+rem #3665 Allow trusted selections in untrusted-mode
This commit is contained in:
parent
55bb2ebcdb
commit
67393c0fb6
7 changed files with 173 additions and 46 deletions
|
|
@ -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::
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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::
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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 ⇒
|
||||||
|
|
|
||||||
|
|
@ -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 {
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue