+rem #3665 Allow trusted selections in untrusted-mode

This commit is contained in:
Patrik Nordwall 2013-11-06 14:34:07 +01:00
parent 55bb2ebcdb
commit 67393c0fb6
7 changed files with 173 additions and 46 deletions

View file

@ -479,6 +479,14 @@ a denial of service attack). :class:`PossiblyHarmful` covers the predefined
messages like :class:`PoisonPill` and :class:`Kill`, but it can also be added messages like :class:`PoisonPill` and :class:`Kill`, but it can also be added
as a marker trait to user-defined messages. as a marker trait to user-defined messages.
Messages sent with actor selection are by default discarded in untrusted mode, but
permission to receive actor selection messages can be granted to specific actors
defined in configuration::
akka.remote.trusted-selection-paths = ["/user/receptionist", "/user/namingService"]
The actual message must still not be of type :class:`PossiblyHarmful`.
In summary, the following operations are ignored by a system configured in In summary, the following operations are ignored by a system configured in
untrusted mode when incoming via the remoting layer: untrusted mode when incoming via the remoting layer:
@ -487,6 +495,7 @@ untrusted mode when incoming via the remoting layer:
* ``system.stop()``, :class:`PoisonPill`, :class:`Kill` * ``system.stop()``, :class:`PoisonPill`, :class:`Kill`
* sending any message which extends from the :class:`PossiblyHarmful` marker * sending any message which extends from the :class:`PossiblyHarmful` marker
interface, which includes :class:`Terminated` interface, which includes :class:`Terminated`
* messages sent with actor selection, unless destination defined in ``trusted-selection-paths``.
.. note:: .. note::

View file

@ -480,6 +480,14 @@ a denial of service attack). :class:`PossiblyHarmful` covers the predefined
messages like :class:`PoisonPill` and :class:`Kill`, but it can also be added messages like :class:`PoisonPill` and :class:`Kill`, but it can also be added
as a marker trait to user-defined messages. as a marker trait to user-defined messages.
Messages sent with actor selection are by default discarded in untrusted mode, but
permission to receive actor selection messages can be granted to specific actors
defined in configuration::
akka.remote.trusted-selection-paths = ["/user/receptionist", "/user/namingService"]
The actual message must still not be of type :class:`PossiblyHarmful`.
In summary, the following operations are ignored by a system configured in In summary, the following operations are ignored by a system configured in
untrusted mode when incoming via the remoting layer: untrusted mode when incoming via the remoting layer:
@ -488,6 +496,7 @@ untrusted mode when incoming via the remoting layer:
* ``system.stop()``, :class:`PoisonPill`, :class:`Kill` * ``system.stop()``, :class:`PoisonPill`, :class:`Kill`
* sending any message which extends from the :class:`PossiblyHarmful` marker * sending any message which extends from the :class:`PossiblyHarmful` marker
interface, which includes :class:`Terminated` interface, which includes :class:`Terminated`
* messages sent with actor selection, unless destination defined in ``trusted-selection-paths``.
.. note:: .. note::

View file

@ -98,6 +98,12 @@ akka {
# system messages to be send by clients, e.g. messages like 'Create', # system messages to be send by clients, e.g. messages like 'Create',
# 'Suspend', 'Resume', 'Terminate', 'Supervise', 'Link' etc. # 'Suspend', 'Resume', 'Terminate', 'Supervise', 'Link' etc.
untrusted-mode = off untrusted-mode = off
# When 'untrusted-mode=on' inbound actor selections are by default discarded.
# Actors with paths defined in this white list are granted permission to receive actor
# selections messages.
# E.g. trusted-selection-paths = ["/user/receptionist", "/user/namingService"]
trusted-selection-paths = []
# Should the remote server require that its peers share the same # Should the remote server require that its peers share the same
# secure-cookie (defined in the 'remote' section)? Secure cookies are passed # secure-cookie (defined in the 'remote' section)? Secure cookies are passed

View file

@ -71,13 +71,19 @@ private[remote] class DefaultMessageDispatcher(private val system: ExtendedActor
case l @ (_: LocalRef | _: RepointableRef) if l.isLocal case l @ (_: LocalRef | _: RepointableRef) if l.isLocal
if (LogReceive) log.debug("received local message {}", msgLog) if (LogReceive) log.debug("received local message {}", msgLog)
payload match { payload match {
case msg: PossiblyHarmful if UntrustedMode
log.debug("operating in UntrustedMode, dropping inbound PossiblyHarmful message of type {}", msg.getClass)
case msg: SystemMessage l.sendSystemMessage(msg)
case sel: ActorSelectionMessage case sel: ActorSelectionMessage
// run the receive logic for ActorSelectionMessage here to make sure it is not stuck on busy user actor if (UntrustedMode && (!TrustedSelectionPaths.contains(sel.elements.mkString("/", "/", "")) ||
ActorSelection.deliverSelection(l, sender, sel) sel.msg.isInstanceOf[PossiblyHarmful] || l != provider.rootGuardian))
case msg l.!(msg)(sender) log.debug("operating in UntrustedMode, dropping inbound actor selection to [{}], " +
"allow it by adding the path to 'akka.remote.trusted-selection-paths' configuration",
sel.elements.mkString("/", "/", ""))
else
// run the receive logic for ActorSelectionMessage here to make sure it is not stuck on busy user actor
ActorSelection.deliverSelection(l, sender, sel)
case msg: PossiblyHarmful if UntrustedMode
log.debug("operating in UntrustedMode, dropping inbound PossiblyHarmful message of type [{}]", msg.getClass.getName)
case msg: SystemMessage l.sendSystemMessage(msg)
case msg l.!(msg)(sender)
} }
case r @ (_: RemoteRef | _: RepointableRef) if !r.isLocal && !UntrustedMode case r @ (_: RemoteRef | _: RepointableRef) if !r.isLocal && !UntrustedMode

View file

@ -25,6 +25,9 @@ final class RemoteSettings(val config: Config) {
val UntrustedMode: Boolean = getBoolean("akka.remote.untrusted-mode") val UntrustedMode: Boolean = getBoolean("akka.remote.untrusted-mode")
val TrustedSelectionPaths: Set[String] =
immutableSeq(getStringList("akka.remote.trusted-selection-paths")).toSet
val RemoteLifecycleEventsLogLevel: LogLevel = getString("akka.remote.log-remote-lifecycle-events").toLowerCase() match { val RemoteLifecycleEventsLogLevel: LogLevel = getString("akka.remote.log-remote-lifecycle-events").toLowerCase() match {
case "on" Logging.DebugLevel case "on" Logging.DebugLevel
case other Logging.levelFor(other) match { case other Logging.levelFor(other) match {

View file

@ -27,6 +27,7 @@ class RemoteConfigSpec extends AkkaSpec(
LogReceive must be(false) LogReceive must be(false)
LogSend must be(false) LogSend must be(false)
UntrustedMode must be(false) UntrustedMode must be(false)
TrustedSelectionPaths must be(Set.empty[String])
LogRemoteLifecycleEvents must be(true) LogRemoteLifecycleEvents must be(true)
ShutdownTimeout.duration must be(10 seconds) ShutdownTimeout.duration must be(10 seconds)
FlushWait must be(2 seconds) FlushWait must be(2 seconds)

View file

@ -4,85 +4,178 @@
package akka.remote package akka.remote
import scala.concurrent.duration._
import com.typesafe.config.ConfigFactory
import akka.actor.Actor
import akka.actor.ActorIdentity
import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.Deploy
import akka.actor.ExtendedActorSystem
import akka.actor.Identify
import akka.actor.PoisonPill
import akka.actor.Props
import akka.actor.RootActorPath
import akka.actor.Terminated
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender import akka.testkit.ImplicitSender
import akka.actor.ActorSystem import akka.testkit.TestProbe
import com.typesafe.config.ConfigFactory import akka.actor.ActorSelection
import akka.actor.ExtendedActorSystem
import akka.actor.RootActorPath
import akka.testkit.EventFilter
import akka.testkit.TestEvent import akka.testkit.TestEvent
import akka.actor.Props
import akka.actor.Actor
import akka.event.Logging import akka.event.Logging
import org.scalatest.junit.JUnitRunner import akka.testkit.EventFilter
import org.junit.runner.RunWith
import akka.actor.Terminated object UntrustedSpec {
import scala.concurrent.duration._ case class IdentifyReq(path: String)
import akka.actor.PoisonPill case class StopChild(name: String)
import akka.actor.Deploy
class Receptionist(testActor: ActorRef) extends Actor {
context.actorOf(Props(classOf[Child], testActor), "child1")
context.actorOf(Props(classOf[Child], testActor), "child2")
context.actorOf(Props(classOf[FakeUser], testActor), "user")
def receive = {
case IdentifyReq(path) context.actorSelection(path).tell(Identify(None), sender)
case StopChild(name) context.child(name) foreach context.stop
case msg testActor forward msg
}
}
class Child(testActor: ActorRef) extends Actor {
override def postStop(): Unit = {
testActor ! s"${self.path.name} stopped"
}
def receive = {
case msg testActor forward msg
}
}
class FakeUser(testActor: ActorRef) extends Actor {
context.actorOf(Props(classOf[Child], testActor), "receptionist")
def receive = {
case msg testActor forward msg
}
}
}
@RunWith(classOf[JUnitRunner])
class UntrustedSpec extends AkkaSpec(""" class UntrustedSpec extends AkkaSpec("""
akka.actor.provider = akka.remote.RemoteActorRefProvider akka.actor.provider = akka.remote.RemoteActorRefProvider
akka.remote.untrusted-mode = on akka.remote.untrusted-mode = on
akka.remote.trusted-selection-paths = ["/user/receptionist", ]
akka.remote.netty.tcp.port = 0 akka.remote.netty.tcp.port = 0
akka.loglevel = DEBUG akka.loglevel = DEBUG
""") with ImplicitSender { """) with ImplicitSender {
val other = ActorSystem("UntrustedSpec-client", ConfigFactory.parseString(""" import UntrustedSpec._
val client = ActorSystem("UntrustedSpec-client", ConfigFactory.parseString("""
akka.actor.provider = akka.remote.RemoteActorRefProvider akka.actor.provider = akka.remote.RemoteActorRefProvider
akka.remote.netty.tcp.port = 0 akka.remote.netty.tcp.port = 0
""")) """))
val addr = system.asInstanceOf[ExtendedActorSystem].provider.asInstanceOf[RemoteActorRefProvider].transport.addresses.head val addr = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress
val target1 = other.actorFor(RootActorPath(addr) / "remote")
val target2 = other.actorFor(RootActorPath(addr) / testActor.path.elements) val receptionist = system.actorOf(Props(classOf[Receptionist], testActor), "receptionist")
lazy val remoteDaemon = {
{
val p = TestProbe()(client)
client.actorSelection(RootActorPath(addr) / receptionist.path.elements).tell(IdentifyReq("/remote"), p.ref)
p.expectMsgType[ActorIdentity].ref.get
}
}
lazy val target2 = {
val p = TestProbe()(client)
client.actorSelection(RootActorPath(addr) / receptionist.path.elements).tell(
IdentifyReq("child2"), p.ref)
p.expectMsgType[ActorIdentity].ref.get
}
override def afterTermination() { override def afterTermination() {
shutdown(other) shutdown(client)
} }
// need to enable debug log-level without actually printing those messages // need to enable debug log-level without actually printing those messages
system.eventStream.publish(TestEvent.Mute(EventFilter.debug())) 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 _
}
}).withDeploy(Deploy.local), "debugSniffer"), classOf[Logging.Debug])
"UntrustedMode" must { "UntrustedMode" must {
"allow actor selection to configured white list" in {
val sel = client.actorSelection(RootActorPath(addr) / receptionist.path.elements)
sel ! "hello"
expectMsg("hello")
}
"discard harmful messages to /remote" in { "discard harmful messages to /remote" in {
target1 ! "hello" val logProbe = TestProbe()
expectMsgType[Logging.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" logProbe.ref ! d
case _
}
}).withDeploy(Deploy.local), "debugSniffer"), classOf[Logging.Debug])
remoteDaemon ! "hello"
logProbe.expectMsgType[Logging.Debug]
} }
"discard harmful messages to testActor" in { "discard harmful messages to testActor" in {
target2 ! Terminated(target1)(existenceConfirmed = true, addressTerminated = false) target2 ! Terminated(remoteDaemon)(existenceConfirmed = true, addressTerminated = false)
expectMsgType[Logging.Debug]
target2 ! PoisonPill target2 ! PoisonPill
expectMsgType[Logging.Debug] client.stop(target2)
other.stop(target2)
expectMsgType[Logging.Debug]
target2 ! "blech" target2 ! "blech"
expectMsg("blech") expectMsg("blech")
} }
"discard watch messages" in { "discard watch messages" in {
other.actorOf(Props(new Actor { client.actorOf(Props(new Actor {
context.watch(target2) context.watch(target2)
def receive = { def receive = {
case x testActor forward x case x testActor forward x
} }
}).withDeploy(Deploy.local)) }).withDeploy(Deploy.local))
within(1.second) { receptionist ! StopChild("child2")
expectMsgType[Logging.Debug] expectMsg("child2 stopped")
expectNoMsg // no Terminated msg, since watch was discarded
} expectNoMsg(1.second)
}
"discard actor selection" in {
val sel = client.actorSelection(RootActorPath(addr) / testActor.path.elements)
sel ! "hello"
expectNoMsg(1.second)
}
"discard actor selection with non root anchor" in {
val p = TestProbe()(client)
client.actorSelection(RootActorPath(addr) / receptionist.path.elements).tell(
Identify(None), p.ref)
val clientReceptionistRef = p.expectMsgType[ActorIdentity].ref.get
val sel = ActorSelection(clientReceptionistRef, receptionist.path.elements.mkString("/", "/", ""))
sel ! "hello"
expectNoMsg(1.second)
}
"discard actor selection to child of matching white list" in {
val sel = client.actorSelection(RootActorPath(addr) / receptionist.path.elements / "child1")
sel ! "hello"
expectNoMsg(1.second)
}
"discard actor selection with wildcard" in {
val sel = client.actorSelection(RootActorPath(addr) / receptionist.path.elements / "*")
sel ! "hello"
expectNoMsg(1.second)
}
"discard actor selection containing harmful message" in {
val sel = client.actorSelection(RootActorPath(addr) / receptionist.path.elements)
sel ! PoisonPill
expectNoMsg(1.second)
} }
} }