double wildcard for actor deployment config #19872
This commit is contained in:
parent
22d669f7f0
commit
05207a1cf6
8 changed files with 203 additions and 52 deletions
|
|
@ -60,6 +60,12 @@ object DeployerSpec {
|
|||
"/*/some" {
|
||||
router = scatter-gather-pool
|
||||
}
|
||||
"/double/**" {
|
||||
router = random-pool
|
||||
}
|
||||
"/double/more/**" {
|
||||
router = round-robin-pool
|
||||
}
|
||||
}
|
||||
""", ConfigParseOptions.defaults)
|
||||
|
||||
|
|
@ -74,7 +80,7 @@ class DeployerSpec extends AkkaSpec(DeployerSpec.deployerConf) {
|
|||
|
||||
"be able to parse 'akka.actor.deployment._' with all default values" in {
|
||||
val service = "/service1"
|
||||
val deployment = system.asInstanceOf[ActorSystemImpl].provider.deployer.lookup(service.split("/").drop(1))
|
||||
val deployment = system.asInstanceOf[ExtendedActorSystem].provider.deployer.lookup(service.split("/").drop(1))
|
||||
|
||||
deployment should ===(Some(
|
||||
Deploy(
|
||||
|
|
@ -88,13 +94,13 @@ class DeployerSpec extends AkkaSpec(DeployerSpec.deployerConf) {
|
|||
|
||||
"use None deployment for undefined service" in {
|
||||
val service = "/undefined"
|
||||
val deployment = system.asInstanceOf[ActorSystemImpl].provider.deployer.lookup(service.split("/").drop(1))
|
||||
val deployment = system.asInstanceOf[ExtendedActorSystem].provider.deployer.lookup(service.split("/").drop(1))
|
||||
deployment should ===(None)
|
||||
}
|
||||
|
||||
"be able to parse 'akka.actor.deployment._' with dispatcher config" in {
|
||||
val service = "/service3"
|
||||
val deployment = system.asInstanceOf[ActorSystemImpl].provider.deployer.lookup(service.split("/").drop(1))
|
||||
val deployment = system.asInstanceOf[ExtendedActorSystem].provider.deployer.lookup(service.split("/").drop(1))
|
||||
|
||||
deployment should ===(Some(
|
||||
Deploy(
|
||||
|
|
@ -108,7 +114,7 @@ class DeployerSpec extends AkkaSpec(DeployerSpec.deployerConf) {
|
|||
|
||||
"be able to parse 'akka.actor.deployment._' with mailbox config" in {
|
||||
val service = "/service4"
|
||||
val deployment = system.asInstanceOf[ActorSystemImpl].provider.deployer.lookup(service.split("/").drop(1))
|
||||
val deployment = system.asInstanceOf[ExtendedActorSystem].provider.deployer.lookup(service.split("/").drop(1))
|
||||
|
||||
deployment should ===(Some(
|
||||
Deploy(
|
||||
|
|
@ -186,8 +192,15 @@ class DeployerSpec extends AkkaSpec(DeployerSpec.deployerConf) {
|
|||
assertRouting("/somewildcardmatch/some", ScatterGatherFirstCompletedPool(nrOfInstances = 1, within = 2 seconds), "/*/some")
|
||||
}
|
||||
|
||||
"be able to use double wildcards" in {
|
||||
assertRouting("/double/wildcardmatch", RandomPool(1), "/double/**")
|
||||
assertRouting("/double/wildcardmatch/anothermatch", RandomPool(1), "/double/**")
|
||||
assertRouting("/double/more/anothermatch", RoundRobinPool(1), "/double/more/**")
|
||||
assertNoRouting("/double")
|
||||
}
|
||||
|
||||
"have correct router mappings" in {
|
||||
val mapping = system.asInstanceOf[ActorSystemImpl].provider.deployer.routerTypeMapping
|
||||
val mapping = system.asInstanceOf[ExtendedActorSystem].provider.deployer.routerTypeMapping
|
||||
mapping("from-code") should ===(classOf[akka.routing.NoRouter].getName)
|
||||
mapping("round-robin-pool") should ===(classOf[akka.routing.RoundRobinPool].getName)
|
||||
mapping("round-robin-group") should ===(classOf[akka.routing.RoundRobinGroup].getName)
|
||||
|
|
@ -203,8 +216,13 @@ class DeployerSpec extends AkkaSpec(DeployerSpec.deployerConf) {
|
|||
mapping("consistent-hashing-group") should ===(classOf[akka.routing.ConsistentHashingGroup].getName)
|
||||
}
|
||||
|
||||
def assertNoRouting(service: String): Unit = {
|
||||
val deployment = system.asInstanceOf[ExtendedActorSystem].provider.deployer.lookup(service.split("/").drop(1))
|
||||
deployment shouldNot be(defined)
|
||||
}
|
||||
|
||||
def assertRouting(service: String, expected: RouterConfig, expectPath: String): Unit = {
|
||||
val deployment = system.asInstanceOf[ActorSystemImpl].provider.deployer.lookup(service.split("/").drop(1))
|
||||
val deployment = system.asInstanceOf[ExtendedActorSystem].provider.deployer.lookup(service.split("/").drop(1))
|
||||
deployment.map(_.path).getOrElse("NOT FOUND") should ===(expectPath)
|
||||
deployment.get.routerConfig.getClass should ===(expected.getClass)
|
||||
deployment.get.scope should ===(NoScopeGiven)
|
||||
|
|
|
|||
|
|
@ -0,0 +1,67 @@
|
|||
/**
|
||||
* Copyright (C) 2016 Lightbend Inc. <http://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.util
|
||||
|
||||
import org.scalatest.{ Matchers, WordSpec }
|
||||
|
||||
class WildcardIndexSpec extends WordSpec with Matchers {
|
||||
|
||||
"wildcard index" must {
|
||||
"allow to insert elements using Arrays of strings" in {
|
||||
emptyIndex.insert(Array("a", "b"), 1) shouldBe a[WildcardIndex[_]]
|
||||
emptyIndex.insert(Array("a"), 1) shouldBe a[WildcardIndex[_]]
|
||||
emptyIndex.insert(Array.empty[String], 1) shouldBe a[WildcardIndex[_]]
|
||||
}
|
||||
|
||||
"allow to find inserted elements" in {
|
||||
val tree = emptyIndex.insert(Array("a"), 1).insert(Array("a", "b"), 2).insert(Array("a", "c"), 3)
|
||||
tree.find(Array("a", "b")).get shouldBe 2
|
||||
tree.find(Array("a")).get shouldBe 1
|
||||
tree.find(Array("x")) shouldBe None
|
||||
tree.find(Array.empty[String]) shouldBe None
|
||||
}
|
||||
|
||||
"match all elements in the subArray when it contains a wildcard" in {
|
||||
val tree1 = emptyIndex.insert(Array("a"), 1).insert(Array("a", "*"), 1)
|
||||
tree1.find(Array("z")) shouldBe None
|
||||
tree1.find(Array("a")).get shouldBe 1
|
||||
tree1.find(Array("a", "b")).get shouldBe 1
|
||||
tree1.find(Array("a", "x")).get shouldBe 1
|
||||
|
||||
val tree2 = emptyIndex.insert(Array("a", "*"), 1).insert(Array("a", "*", "c"), 2)
|
||||
tree2.find(Array("z")) shouldBe None
|
||||
tree2.find(Array("a", "b")).get shouldBe 1
|
||||
tree2.find(Array("a", "x")).get shouldBe 1
|
||||
tree2.find(Array("a", "x", "c")).get shouldBe 2
|
||||
tree2.find(Array("a", "x", "y")) shouldBe None
|
||||
}
|
||||
|
||||
"never find anything when emptyIndex" in {
|
||||
emptyIndex.find(Array("a")) shouldBe None
|
||||
emptyIndex.find(Array("a", "b")) shouldBe None
|
||||
emptyIndex.find(Array.empty[String]) shouldBe None
|
||||
}
|
||||
|
||||
"match all remaining elements when it contains a terminal double wildcard" in {
|
||||
val tree1 = emptyIndex.insert(Array("a", "**"), 1)
|
||||
tree1.find(Array("z")) shouldBe None
|
||||
tree1.find(Array("a", "b")).get shouldBe 1
|
||||
tree1.find(Array("a", "x")).get shouldBe 1
|
||||
tree1.find(Array("a", "x", "y")).get shouldBe 1
|
||||
|
||||
val tree2 = emptyIndex.insert(Array("**"), 1)
|
||||
tree2.find(Array("anything", "I", "want")).get shouldBe 1
|
||||
tree2.find(Array("anything")).get shouldBe 1
|
||||
}
|
||||
|
||||
"ignore non-terminal double wildcards" in {
|
||||
val tree = emptyIndex.insert(Array("a", "**", "c"), 1)
|
||||
tree.find(Array("a", "x", "y", "c")) shouldBe None
|
||||
tree.find(Array("a", "x", "y")) shouldBe None
|
||||
}
|
||||
}
|
||||
|
||||
private val emptyIndex = WildcardIndex[Int]()
|
||||
}
|
||||
|
|
@ -7,7 +7,7 @@ package akka.actor
|
|||
import java.util.concurrent.atomic.AtomicReference
|
||||
|
||||
import akka.routing._
|
||||
import akka.util.WildcardTree
|
||||
import akka.util.WildcardIndex
|
||||
import com.typesafe.config._
|
||||
|
||||
import scala.annotation.tailrec
|
||||
|
|
@ -132,7 +132,7 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce
|
|||
import scala.collection.JavaConverters._
|
||||
|
||||
private val resizerEnabled: Config = ConfigFactory.parseString("resizer.enabled=on")
|
||||
private val deployments = new AtomicReference(WildcardTree[Deploy]())
|
||||
private val deployments = new AtomicReference(WildcardIndex[Deploy]())
|
||||
private val config = settings.config.getConfig("akka.actor.deployment")
|
||||
protected val default = config.getConfig("default")
|
||||
val routerTypeMapping: Map[String, String] =
|
||||
|
|
@ -146,20 +146,18 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce
|
|||
case _ ⇒ None
|
||||
} foreach deploy
|
||||
|
||||
def lookup(path: ActorPath): Option[Deploy] = lookup(path.elements.drop(1).iterator)
|
||||
def lookup(path: ActorPath): Option[Deploy] = lookup(path.elements.drop(1))
|
||||
|
||||
def lookup(path: Iterable[String]): Option[Deploy] = lookup(path.iterator)
|
||||
|
||||
def lookup(path: Iterator[String]): Option[Deploy] = deployments.get().find(path).data
|
||||
def lookup(path: Iterable[String]): Option[Deploy] = deployments.get().find(path)
|
||||
|
||||
def deploy(d: Deploy): Unit = {
|
||||
@tailrec def add(path: Array[String], d: Deploy, w: WildcardTree[Deploy] = deployments.get): Unit = {
|
||||
for (i ← 0 until path.length) path(i) match {
|
||||
@tailrec def add(path: Array[String], d: Deploy, w: WildcardIndex[Deploy] = deployments.get): Unit = {
|
||||
for (i ← path.indices) path(i) match {
|
||||
case "" ⇒ throw new InvalidActorNameException(s"Actor name in deployment [${d.path}] must not be empty")
|
||||
case el ⇒ ActorPath.validatePathElement(el, fullPath = d.path)
|
||||
}
|
||||
|
||||
if (!deployments.compareAndSet(w, w.insert(path.iterator, d))) add(path, d)
|
||||
if (!deployments.compareAndSet(w, w.insert(path, d))) add(path, d)
|
||||
}
|
||||
|
||||
add(d.path.split("/").drop(1), d)
|
||||
|
|
|
|||
81
akka-actor/src/main/scala/akka/util/WildcardIndex.scala
Normal file
81
akka-actor/src/main/scala/akka/util/WildcardIndex.scala
Normal file
|
|
@ -0,0 +1,81 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2016 Lightbend Inc. <http://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.util
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable.HashMap
|
||||
|
||||
private[akka] final case class WildcardIndex[T](wildcardTree: WildcardTree[T] = WildcardTree[T](), doubleWildcardTree: WildcardTree[T] = WildcardTree[T]()) {
|
||||
|
||||
val empty = WildcardTree[T]()
|
||||
|
||||
def insert(elems: Array[String], d: T): WildcardIndex[T] = elems.lastOption match {
|
||||
case Some("**") ⇒ copy(doubleWildcardTree = doubleWildcardTree.insert(elems.iterator, d))
|
||||
case Some(_) ⇒ copy(wildcardTree = wildcardTree.insert(elems.iterator, d))
|
||||
case _ ⇒ this
|
||||
}
|
||||
|
||||
def find(elems: Iterable[String]): Option[T] =
|
||||
(if (wildcardTree.isEmpty) {
|
||||
if (doubleWildcardTree.isEmpty) {
|
||||
empty
|
||||
} else {
|
||||
doubleWildcardTree.findWithTerminalDoubleWildcard(elems.iterator)
|
||||
}
|
||||
} else {
|
||||
val withSingleWildcard = wildcardTree.findWithSingleWildcard(elems.iterator)
|
||||
if (withSingleWildcard.isEmpty) {
|
||||
doubleWildcardTree.findWithTerminalDoubleWildcard(elems.iterator)
|
||||
} else {
|
||||
withSingleWildcard
|
||||
}
|
||||
}).data
|
||||
|
||||
}
|
||||
|
||||
private[akka] object WildcardTree {
|
||||
private val empty = new WildcardTree[Nothing]()
|
||||
def apply[T](): WildcardTree[T] = empty.asInstanceOf[WildcardTree[T]]
|
||||
}
|
||||
|
||||
private[akka] final case class WildcardTree[T](data: Option[T] = None, children: Map[String, WildcardTree[T]] = HashMap[String, WildcardTree[T]]()) {
|
||||
|
||||
lazy val isEmpty: Boolean = data.isEmpty && children.isEmpty
|
||||
|
||||
def insert(elems: Iterator[String], d: T): WildcardTree[T] =
|
||||
if (!elems.hasNext) {
|
||||
copy(data = Some(d))
|
||||
} else {
|
||||
val e = elems.next()
|
||||
copy(children = children.updated(e, children.getOrElse(e, WildcardTree[T]()).insert(elems, d)))
|
||||
}
|
||||
|
||||
@tailrec def findWithSingleWildcard(elems: Iterator[String]): WildcardTree[T] =
|
||||
if (!elems.hasNext) this
|
||||
else {
|
||||
children.get(elems.next()) match {
|
||||
case Some(branch) ⇒ branch.findWithSingleWildcard(elems)
|
||||
case None ⇒ children.get("*") match {
|
||||
case Some(branch) ⇒ branch.findWithSingleWildcard(elems)
|
||||
case None ⇒ WildcardTree[T]()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@tailrec def findWithTerminalDoubleWildcard(elems: Iterator[String], alt: WildcardTree[T] = WildcardTree[T]()): WildcardTree[T] = {
|
||||
if (!elems.hasNext) this
|
||||
else {
|
||||
val newAlt = children.getOrElse("**", alt)
|
||||
children.get(elems.next()) match {
|
||||
case Some(branch) ⇒ branch.findWithTerminalDoubleWildcard(elems, newAlt)
|
||||
case None ⇒ children.get("*") match {
|
||||
case Some(branch) ⇒ branch.findWithTerminalDoubleWildcard(elems, newAlt)
|
||||
case None ⇒ newAlt
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -1,32 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2016 Lightbend Inc. <http://www.lightbend.com>
|
||||
*/
|
||||
|
||||
package akka.util
|
||||
|
||||
import annotation.tailrec
|
||||
import collection.immutable.HashMap
|
||||
|
||||
private[akka] object WildcardTree {
|
||||
private val empty = new WildcardTree[Nothing]()
|
||||
def apply[T](): WildcardTree[T] = empty.asInstanceOf[WildcardTree[T]]
|
||||
}
|
||||
private[akka] final case class WildcardTree[T](data: Option[T] = None, children: Map[String, WildcardTree[T]] = HashMap[String, WildcardTree[T]]()) {
|
||||
|
||||
def insert(elems: Iterator[String], d: T): WildcardTree[T] =
|
||||
if (!elems.hasNext) {
|
||||
copy(data = Some(d))
|
||||
} else {
|
||||
val e = elems.next()
|
||||
copy(children = children.updated(e, children.get(e).getOrElse(WildcardTree()).insert(elems, d)))
|
||||
}
|
||||
|
||||
@tailrec final def find(elems: Iterator[String]): WildcardTree[T] =
|
||||
if (!elems.hasNext) this
|
||||
else {
|
||||
(children.get(elems.next()) orElse children.get("*")) match {
|
||||
case Some(branch) ⇒ branch.find(elems)
|
||||
case None ⇒ WildcardTree()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -46,6 +46,12 @@ class ConfigDocSpec extends WordSpec with Matchers {
|
|||
"/actorC/*" {
|
||||
dispatcher = my-dispatcher
|
||||
}
|
||||
|
||||
# all descendants of '/user/actorC' (direct children, and their children recursively)
|
||||
# have a dedicated dispatcher
|
||||
"/actorC/**" {
|
||||
dispatcher = my-dispatcher
|
||||
}
|
||||
|
||||
# '/user/actorD/actorE' has a special priority mailbox
|
||||
/actorD/actorE {
|
||||
|
|
|
|||
|
|
@ -413,10 +413,18 @@ topics. An example may look like this:
|
|||
|
||||
You can use asterisks as wildcard matches for the actor path sections, so you could specify:
|
||||
``/*/sampleActor`` and that would match all ``sampleActor`` on that level in the hierarchy.
|
||||
You can also use wildcard in the last position to match all actors at a certain level:
|
||||
``/someParent/*``. Non-wildcard matches always have higher priority to match than wildcards, so:
|
||||
``/foo/bar`` is considered **more specific** than ``/foo/*`` and only the highest priority match is used.
|
||||
Please note that it **cannot** be used to partially match section, like this: ``/foo*/bar``, ``/f*o/bar`` etc.
|
||||
In addition, please note:
|
||||
|
||||
- you can also use wildcards in the last position to match all actors at a certain level: ``/someParent/*``
|
||||
- you can use double-wildcards in the last position to match all child actors and their children
|
||||
recursively: ``/someParent/**``
|
||||
- non-wildcard matches always have higher priority to match than wildcards, and single wildcard matches
|
||||
have higher priority than double-wildcards, so: ``/foo/bar`` is considered **more specific** than
|
||||
``/foo/*``, which is considered **more specific** than ``/foo/**``. Only the highest priority match is used
|
||||
- wildcards **cannot** be used to partially match section, like this: ``/foo*/bar``, ``/f*o/bar`` etc.
|
||||
|
||||
.. note::
|
||||
Double-wildcards can only be placed in the last position.
|
||||
|
||||
Listing of the Reference Configuration
|
||||
--------------------------------------
|
||||
|
|
|
|||
|
|
@ -934,7 +934,12 @@ object MiMa extends AutoPlugin {
|
|||
|
||||
// #20630 corrected return types of java methods
|
||||
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.stream.javadsl.RunnableGraph#RunnableGraphAdapter.named"),
|
||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.javadsl.RunnableGraph.withAttributes")
|
||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.javadsl.RunnableGraph.withAttributes"),
|
||||
|
||||
// #19872 double wildcard for actor deployment config
|
||||
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.actor.Deployer.lookup"),
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.util.WildcardTree.apply"),
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.util.WildcardTree.find")
|
||||
)
|
||||
)
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue