Merge with latest changes on akka/master and simplify OSGi configuration section
Conflicts: project/AkkaBuild.scala
This commit is contained in:
commit
e24f7077ec
264 changed files with 9794 additions and 3334 deletions
|
|
@ -0,0 +1,77 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.actor
|
||||
|
||||
import akka.testkit._
|
||||
import akka.testkit.DefaultTimeout
|
||||
import akka.testkit.TestEvent._
|
||||
import akka.util.duration._
|
||||
import akka.routing._
|
||||
import org.scalatest.BeforeAndAfterEach
|
||||
import akka.ConfigurationException
|
||||
|
||||
object ActorConfigurationVerificationSpec {
|
||||
|
||||
class TestActor extends Actor {
|
||||
def receive: Receive = {
|
||||
case _ ⇒
|
||||
}
|
||||
}
|
||||
|
||||
val config = """
|
||||
balancing-dispatcher {
|
||||
type = BalancingDispatcher
|
||||
throughput = 1
|
||||
}
|
||||
pinned-dispatcher {
|
||||
executor = "thread-pool-executor"
|
||||
type = PinnedDispatcher
|
||||
}
|
||||
"""
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class ActorConfigurationVerificationSpec extends AkkaSpec(ActorConfigurationVerificationSpec.config) with DefaultTimeout with BeforeAndAfterEach {
|
||||
import ActorConfigurationVerificationSpec._
|
||||
|
||||
override def atStartup {
|
||||
system.eventStream.publish(Mute(EventFilter[ConfigurationException]("")))
|
||||
}
|
||||
|
||||
"An Actor configured with a BalancingDispatcher" must {
|
||||
"fail verification with a ConfigurationException if also configured with a RoundRobinRouter" in {
|
||||
intercept[ConfigurationException] {
|
||||
system.actorOf(Props[TestActor].withDispatcher("balancing-dispatcher").withRouter(RoundRobinRouter(2)))
|
||||
}
|
||||
}
|
||||
"fail verification with a ConfigurationException if also configured with a BroadcastRouter" in {
|
||||
intercept[ConfigurationException] {
|
||||
system.actorOf(Props[TestActor].withDispatcher("balancing-dispatcher").withRouter(BroadcastRouter(2)))
|
||||
}
|
||||
}
|
||||
"fail verification with a ConfigurationException if also configured with a RandomRouter" in {
|
||||
intercept[ConfigurationException] {
|
||||
system.actorOf(Props[TestActor].withDispatcher("balancing-dispatcher").withRouter(RandomRouter(2)))
|
||||
}
|
||||
}
|
||||
"fail verification with a ConfigurationException if also configured with a SmallestMailboxRouter" in {
|
||||
intercept[ConfigurationException] {
|
||||
system.actorOf(Props[TestActor].withDispatcher("balancing-dispatcher").withRouter(SmallestMailboxRouter(2)))
|
||||
}
|
||||
}
|
||||
"fail verification with a ConfigurationException if also configured with a ScatterGatherFirstCompletedRouter" in {
|
||||
intercept[ConfigurationException] {
|
||||
system.actorOf(Props[TestActor].withDispatcher("balancing-dispatcher").withRouter(ScatterGatherFirstCompletedRouter(nrOfInstances = 2, within = 2 seconds)))
|
||||
}
|
||||
}
|
||||
"not fail verification with a ConfigurationException also not configured with a Router" in {
|
||||
system.actorOf(Props[TestActor].withDispatcher("balancing-dispatcher"))
|
||||
}
|
||||
}
|
||||
"An Actor configured with a non-balancing dispatcher" must {
|
||||
"not fail verification with a ConfigurationException if also configured with a Router" in {
|
||||
system.actorOf(Props[TestActor].withDispatcher("pinned-dispatcher").withRouter(RoundRobinRouter(2)))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -13,6 +13,7 @@ import akka.util.duration._
|
|||
import java.util.concurrent.atomic._
|
||||
import akka.dispatch.Await
|
||||
import akka.pattern.ask
|
||||
import java.util.UUID.{ randomUUID ⇒ newUuid }
|
||||
|
||||
object ActorLifeCycleSpec {
|
||||
|
||||
|
|
@ -35,7 +36,7 @@ class ActorLifeCycleSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitS
|
|||
|
||||
"invoke preRestart, preStart, postRestart when using OneForOneStrategy" in {
|
||||
filterException[ActorKilledException] {
|
||||
val id = newUuid().toString
|
||||
val id = newUuid.toString
|
||||
val supervisor = system.actorOf(Props(new Supervisor(
|
||||
OneForOneStrategy(maxNrOfRetries = 3)(List(classOf[Exception])))))
|
||||
val gen = new AtomicInteger(0)
|
||||
|
|
|
|||
|
|
@ -393,7 +393,7 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
|
|||
override def postRestart(reason: Throwable) = latch.countDown()
|
||||
}))
|
||||
|
||||
protected def receive = { case "sendKill" ⇒ ref ! Kill }
|
||||
def receive = { case "sendKill" ⇒ ref ! Kill }
|
||||
}))
|
||||
|
||||
boss ! "sendKill"
|
||||
|
|
|
|||
|
|
@ -131,7 +131,7 @@ class ActorWithStashSpec extends AkkaSpec(ActorWithStashSpec.testConf) with Defa
|
|||
val hasMsgLatch = new TestLatch
|
||||
|
||||
val slaveProps = myProps(new Actor with Stash {
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case "crash" ⇒
|
||||
throw new Exception("Crashing...")
|
||||
|
||||
|
|
|
|||
|
|
@ -147,7 +147,7 @@ class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with Im
|
|||
object Hello
|
||||
object Bye
|
||||
val tester = system.actorOf(Props(new Actor {
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case Hello ⇒ lock ! "hello"
|
||||
case "world" ⇒ answerLatch.open
|
||||
case Bye ⇒ lock ! "bye"
|
||||
|
|
|
|||
|
|
@ -6,10 +6,60 @@ package akka.actor
|
|||
|
||||
import akka.testkit._
|
||||
|
||||
object HotSwapSpec {
|
||||
abstract class Becomer extends Actor {
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class HotSwapSpec extends AkkaSpec with ImplicitSender {
|
||||
import HotSwapSpec.Becomer
|
||||
|
||||
"An Actor" must {
|
||||
"be able to become in its constructor" in {
|
||||
val a = system.actorOf(Props(new Becomer {
|
||||
context.become { case always ⇒ sender ! always }
|
||||
def receive = { case always ⇒ sender ! "FAILURE" }
|
||||
}))
|
||||
a ! "pigdog"
|
||||
expectMsg("pigdog")
|
||||
}
|
||||
|
||||
"be able to become multiple times in its constructor" in {
|
||||
val a = system.actorOf(Props(new Becomer {
|
||||
for (i ← 1 to 4) context.become({ case always ⇒ sender ! i + ":" + always })
|
||||
def receive = { case always ⇒ sender ! "FAILURE" }
|
||||
}))
|
||||
a ! "pigdog"
|
||||
expectMsg("4:pigdog")
|
||||
}
|
||||
|
||||
"be able to become with stacking in its constructor" in {
|
||||
val a = system.actorOf(Props(new Becomer {
|
||||
context.become({ case always ⇒ sender ! "pigdog:" + always; context.unbecome() }, false)
|
||||
def receive = { case always ⇒ sender ! "badass:" + always }
|
||||
}))
|
||||
a ! "pigdog"
|
||||
expectMsg("pigdog:pigdog")
|
||||
a ! "badass"
|
||||
expectMsg("badass:badass")
|
||||
}
|
||||
|
||||
"be able to become, with stacking, multiple times in its constructor" in {
|
||||
val a = system.actorOf(Props(new Becomer {
|
||||
for (i ← 1 to 4) context.become({ case always ⇒ sender ! i + ":" + always; context.unbecome() }, false)
|
||||
def receive = { case always ⇒ sender ! "FAILURE" }
|
||||
}))
|
||||
a ! "pigdog"
|
||||
a ! "pigdog"
|
||||
a ! "pigdog"
|
||||
a ! "pigdog"
|
||||
expectMsg("4:pigdog")
|
||||
expectMsg("3:pigdog")
|
||||
expectMsg("2:pigdog")
|
||||
expectMsg("1:pigdog")
|
||||
}
|
||||
|
||||
"be able to hotswap its behavior with become(..)" in {
|
||||
val a = system.actorOf(Props(new Actor {
|
||||
|
|
@ -30,13 +80,10 @@ class HotSwapSpec extends AkkaSpec with ImplicitSender {
|
|||
val a = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "init" ⇒ sender ! "init"
|
||||
case "swap" ⇒
|
||||
context.become({
|
||||
case "swapped" ⇒
|
||||
sender ! "swapped"
|
||||
case "revert" ⇒
|
||||
context.unbecome()
|
||||
})
|
||||
case "swap" ⇒ context.become({
|
||||
case "swapped" ⇒ sender ! "swapped"
|
||||
case "revert" ⇒ context.unbecome()
|
||||
})
|
||||
}
|
||||
}))
|
||||
|
||||
|
|
|
|||
|
|
@ -22,7 +22,7 @@ class ReceiveTimeoutSpec extends AkkaSpec {
|
|||
val timeoutActor = system.actorOf(Props(new Actor {
|
||||
context.setReceiveTimeout(500 milliseconds)
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case ReceiveTimeout ⇒ timeoutLatch.open
|
||||
}
|
||||
}))
|
||||
|
|
@ -38,7 +38,7 @@ class ReceiveTimeoutSpec extends AkkaSpec {
|
|||
val timeoutActor = system.actorOf(Props(new Actor {
|
||||
context.setReceiveTimeout(500 milliseconds)
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case Tick ⇒ ()
|
||||
case ReceiveTimeout ⇒ timeoutLatch.open
|
||||
}
|
||||
|
|
@ -58,7 +58,7 @@ class ReceiveTimeoutSpec extends AkkaSpec {
|
|||
val timeoutActor = system.actorOf(Props(new Actor {
|
||||
context.setReceiveTimeout(500 milliseconds)
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case Tick ⇒ ()
|
||||
case ReceiveTimeout ⇒
|
||||
count.incrementAndGet
|
||||
|
|
@ -78,7 +78,7 @@ class ReceiveTimeoutSpec extends AkkaSpec {
|
|||
val timeoutLatch = TestLatch()
|
||||
|
||||
val timeoutActor = system.actorOf(Props(new Actor {
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case ReceiveTimeout ⇒ timeoutLatch.open
|
||||
}
|
||||
}))
|
||||
|
|
|
|||
|
|
@ -40,7 +40,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout {
|
|||
|
||||
val slaveProps = Props(new Actor {
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case Ping ⇒ countDownLatch.countDown()
|
||||
case Crash ⇒ throw new Exception("Crashing...")
|
||||
}
|
||||
|
|
@ -83,7 +83,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout {
|
|||
|
||||
val slaveProps = Props(new Actor {
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case Crash ⇒ throw new Exception("Crashing...")
|
||||
}
|
||||
|
||||
|
|
@ -110,7 +110,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout {
|
|||
|
||||
val slaveProps = Props(new Actor {
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case Ping ⇒
|
||||
if (!pingLatch.isOpen) pingLatch.open else secondPingLatch.open
|
||||
case Crash ⇒ throw new Exception("Crashing...")
|
||||
|
|
@ -166,7 +166,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout {
|
|||
|
||||
val slaveProps = Props(new Actor {
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case Ping ⇒ countDownLatch.countDown()
|
||||
case Crash ⇒ throw new Exception("Crashing...")
|
||||
}
|
||||
|
|
@ -221,7 +221,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout {
|
|||
|
||||
val slaveProps = Props(new Actor {
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case Ping ⇒ countDownLatch.countDown()
|
||||
case Crash ⇒ throw new Exception("Crashing...")
|
||||
}
|
||||
|
|
|
|||
|
|
@ -18,7 +18,12 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout
|
|||
}
|
||||
|
||||
override def afterEach {
|
||||
while (cancellables.peek() ne null) { Option(cancellables.poll()).foreach(_.cancel()) }
|
||||
while (cancellables.peek() ne null) {
|
||||
for (c ← Option(cancellables.poll())) {
|
||||
c.cancel()
|
||||
c.isCancelled must be === true
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
"A Scheduler" must {
|
||||
|
|
|
|||
|
|
@ -20,7 +20,7 @@ object SupervisorHierarchySpec {
|
|||
*/
|
||||
class CountDownActor(countDown: CountDownLatch, override val supervisorStrategy: SupervisorStrategy) extends Actor {
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case p: Props ⇒ sender ! context.actorOf(p)
|
||||
}
|
||||
// test relies on keeping children around during restart
|
||||
|
|
@ -67,7 +67,7 @@ class SupervisorHierarchySpec extends AkkaSpec with DefaultTimeout {
|
|||
|
||||
val crasher = context.watch(context.actorOf(Props(new CountDownActor(countDownMessages, SupervisorStrategy.defaultStrategy))))
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case "killCrasher" ⇒ crasher ! Kill
|
||||
case Terminated(_) ⇒ countDownMax.countDown()
|
||||
}
|
||||
|
|
|
|||
|
|
@ -37,7 +37,7 @@ class SupervisorMiscSpec extends AkkaSpec(SupervisorMiscSpec.config) with Defaul
|
|||
|
||||
val workerProps = Props(new Actor {
|
||||
override def postRestart(cause: Throwable) { countDownLatch.countDown() }
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case "status" ⇒ this.sender ! "OK"
|
||||
case _ ⇒ this.context.stop(self)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -400,17 +400,17 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa
|
|||
val a = newTestActor(dispatcher.id)
|
||||
val f1 = a ? Reply("foo")
|
||||
val f2 = a ? Reply("bar")
|
||||
val f3 = try { a ? Interrupt } catch { case ie: InterruptedException ⇒ Promise.failed(ActorInterruptedException(ie)) }
|
||||
val f3 = try { a ? Interrupt } catch { case ie: InterruptedException ⇒ Promise.failed(new ActorInterruptedException(ie)) }
|
||||
val f4 = a ? Reply("foo2")
|
||||
val f5 = try { a ? Interrupt } catch { case ie: InterruptedException ⇒ Promise.failed(ActorInterruptedException(ie)) }
|
||||
val f5 = try { a ? Interrupt } catch { case ie: InterruptedException ⇒ Promise.failed(new ActorInterruptedException(ie)) }
|
||||
val f6 = a ? Reply("bar2")
|
||||
|
||||
assert(Await.result(f1, timeout.duration) === "foo")
|
||||
assert(Await.result(f2, timeout.duration) === "bar")
|
||||
assert(Await.result(f4, timeout.duration) === "foo2")
|
||||
assert(intercept[ActorInterruptedException](Await.result(f3, timeout.duration)).getMessage === "Ping!")
|
||||
assert(intercept[ActorInterruptedException](Await.result(f3, timeout.duration)).getCause.getMessage === "Ping!")
|
||||
assert(Await.result(f6, timeout.duration) === "bar2")
|
||||
assert(intercept[ActorInterruptedException](Await.result(f5, timeout.duration)).getMessage === "Ping!")
|
||||
assert(intercept[ActorInterruptedException](Await.result(f5, timeout.duration)).getCause.getMessage === "Ping!")
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -19,7 +19,7 @@ object EventStreamSpec {
|
|||
loglevel = INFO
|
||||
event-handlers = ["akka.event.EventStreamSpec$MyLog", "%s"]
|
||||
}
|
||||
""".format(Logging.StandardOutLoggerName))
|
||||
""".format(Logging.StandardOutLogger.getClass.getName))
|
||||
|
||||
val configUnhandled = ConfigFactory.parseString("""
|
||||
akka {
|
||||
|
|
|
|||
|
|
@ -7,11 +7,9 @@ package akka.pattern
|
|||
import akka.testkit.AkkaSpec
|
||||
import akka.actor.Props
|
||||
import akka.actor.Actor
|
||||
import akka.actor.ActorTimeoutException
|
||||
import akka.util.Duration
|
||||
import akka.util.duration._
|
||||
import akka.dispatch.{ Future, Promise, Await }
|
||||
import java.lang.IllegalStateException
|
||||
|
||||
object PatternSpec {
|
||||
case class Work(duration: Duration)
|
||||
|
|
@ -41,13 +39,10 @@ class PatternSpec extends AkkaSpec {
|
|||
Await.ready(gracefulStop(target, 1 millis), 1 second)
|
||||
}
|
||||
|
||||
"complete Future with ActorTimeoutException when actor not terminated within timeout" in {
|
||||
"complete Future with AskTimeoutException when actor not terminated within timeout" in {
|
||||
val target = system.actorOf(Props[TargetActor])
|
||||
target ! Work(250 millis)
|
||||
val result = gracefulStop(target, 10 millis)
|
||||
intercept[ActorTimeoutException] {
|
||||
Await.result(result, 200 millis)
|
||||
}
|
||||
intercept[AskTimeoutException] { Await.result(gracefulStop(target, 10 millis), 200 millis) }
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -3,12 +3,11 @@ package akka.performance.workbench
|
|||
import scala.collection.immutable.TreeMap
|
||||
import org.apache.commons.math.stat.descriptive.DescriptiveStatistics
|
||||
import org.scalatest.BeforeAndAfterEach
|
||||
import akka.actor.simpleName
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.actor.ActorSystem
|
||||
import akka.util.Duration
|
||||
import com.typesafe.config.Config
|
||||
import java.util.concurrent.TimeUnit
|
||||
import akka.event.Logging
|
||||
|
||||
abstract class PerformanceSpec(cfg: Config = BenchmarkConfig.config) extends AkkaSpec(cfg) with BeforeAndAfterEach {
|
||||
|
||||
|
|
@ -36,7 +35,7 @@ abstract class PerformanceSpec(cfg: Config = BenchmarkConfig.config) extends Akk
|
|||
}
|
||||
|
||||
def logMeasurement(numberOfClients: Int, durationNs: Long, n: Long) {
|
||||
val name = simpleName(this)
|
||||
val name = Logging.simpleName(this)
|
||||
val durationS = durationNs.toDouble / 1000000000.0
|
||||
|
||||
val stats = Stats(
|
||||
|
|
@ -51,7 +50,7 @@ abstract class PerformanceSpec(cfg: Config = BenchmarkConfig.config) extends Akk
|
|||
}
|
||||
|
||||
def logMeasurement(numberOfClients: Int, durationNs: Long, stat: DescriptiveStatistics) {
|
||||
val name = simpleName(this)
|
||||
val name = Logging.simpleName(this)
|
||||
val durationS = durationNs.toDouble / 1000000000.0
|
||||
|
||||
val percentiles = TreeMap[Int, Long](
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ import java.util.concurrent.atomic.AtomicInteger
|
|||
import org.junit.runner.RunWith
|
||||
|
||||
import akka.actor.{ Props, LocalActorRef, Deploy, Actor, ActorRef }
|
||||
import akka.config.ConfigurationException
|
||||
import akka.ConfigurationException
|
||||
import akka.dispatch.Await
|
||||
import akka.pattern.{ ask, gracefulStop }
|
||||
import akka.testkit.{ TestLatch, ImplicitSender, DefaultTimeout, AkkaSpec }
|
||||
|
|
|
|||
|
|
@ -128,35 +128,6 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with
|
|||
current.routees.size must be(2)
|
||||
}
|
||||
|
||||
"resize when busy" in {
|
||||
|
||||
val busy = new TestLatch(1)
|
||||
|
||||
val resizer = DefaultResizer(
|
||||
lowerBound = 1,
|
||||
upperBound = 3,
|
||||
pressureThreshold = 0,
|
||||
messagesPerResize = 1)
|
||||
|
||||
val router = system.actorOf(Props[BusyActor].withRouter(RoundRobinRouter(resizer = Some(resizer))).withDispatcher("bal-disp"))
|
||||
|
||||
val latch1 = new TestLatch(1)
|
||||
router ! (latch1, busy)
|
||||
Await.ready(latch1, 2 seconds)
|
||||
|
||||
val latch2 = new TestLatch(1)
|
||||
router ! (latch2, busy)
|
||||
Await.ready(latch2, 2 seconds)
|
||||
|
||||
val latch3 = new TestLatch(1)
|
||||
router ! (latch3, busy)
|
||||
Await.ready(latch3, 2 seconds)
|
||||
|
||||
Await.result(router ? CurrentRoutees, 5 seconds).asInstanceOf[RouterRoutees].routees.size must be(3)
|
||||
|
||||
busy.countDown()
|
||||
}
|
||||
|
||||
"grow as needed under pressure" in {
|
||||
// make sure the pool starts at the expected lower limit and grows to the upper as needed
|
||||
// as influenced by the backlog of blocking pooled actors
|
||||
|
|
|
|||
|
|
@ -10,7 +10,7 @@ import akka.testkit._
|
|||
import akka.util.duration._
|
||||
import akka.dispatch.Await
|
||||
import akka.util.Duration
|
||||
import akka.config.ConfigurationException
|
||||
import akka.ConfigurationException
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.pattern.ask
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
|
|
|
|||
|
|
@ -18,7 +18,7 @@
|
|||
* Expert Group and released to the public domain, as explained at
|
||||
* http://creativecommons.org/licenses/publicdomain
|
||||
*/
|
||||
package org.jboss.netty.akka.util.internal;
|
||||
package akka.util.internal;
|
||||
|
||||
import java.util.AbstractCollection;
|
||||
import java.util.AbstractMap;
|
||||
|
|
@ -13,12 +13,10 @@
|
|||
* License for the specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.jboss.netty.akka.util;
|
||||
package akka.util.internal;
|
||||
|
||||
import akka.event.LoggingAdapter;
|
||||
import akka.util.Duration;
|
||||
import org.jboss.netty.akka.util.internal.ConcurrentIdentityHashMap;
|
||||
import org.jboss.netty.akka.util.internal.ReusableIterator;
|
||||
|
||||
import java.util.*;
|
||||
import java.util.concurrent.ThreadFactory;
|
||||
|
|
@ -34,7 +32,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|||
* <h3>Tick Duration</h3>
|
||||
*
|
||||
* As described with 'approximated', this timer does not execute the scheduled
|
||||
* {@link TimerTask} on time. {@link org.jboss.netty.akka.util.HashedWheelTimer}, on every tick, will
|
||||
* {@link TimerTask} on time. {@link HashedWheelTimer}, on every tick, will
|
||||
* check if there are any {@link TimerTask}s behind the schedule and execute
|
||||
* them.
|
||||
* <p>
|
||||
|
|
@ -46,7 +44,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|||
*
|
||||
* <h3>Ticks per Wheel (Wheel Size)</h3>
|
||||
*
|
||||
* {@link org.jboss.netty.akka.util.HashedWheelTimer} maintains a data structure called 'wheel'.
|
||||
* {@link HashedWheelTimer} maintains a data structure called 'wheel'.
|
||||
* To put simply, a wheel is a hash table of {@link TimerTask}s whose hash
|
||||
* function is 'dead line of the task'. The default number of ticks per wheel
|
||||
* (i.e. the size of the wheel) is 512. You could specify a larger value
|
||||
|
|
@ -54,7 +52,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|||
*
|
||||
* <h3>Do not create many instances.</h3>
|
||||
*
|
||||
* {@link org.jboss.netty.akka.util.HashedWheelTimer} creates a new thread whenever it is instantiated and
|
||||
* {@link HashedWheelTimer} creates a new thread whenever it is instantiated and
|
||||
* started. Therefore, you should make sure to create only one instance and
|
||||
* share it across your application. One of the common mistakes, that makes
|
||||
* your application unresponsive, is to create a new instance in
|
||||
|
|
@ -63,7 +61,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|||
*
|
||||
* <h3>Implementation Details</h3>
|
||||
*
|
||||
* {@link org.jboss.netty.akka.util.HashedWheelTimer} is based on
|
||||
* {@link HashedWheelTimer} is based on
|
||||
* <a href="http://cseweb.ucsd.edu/users/varghese/">George Varghese</a> and
|
||||
* Tony Lauck's paper,
|
||||
* <a href="http://cseweb.ucsd.edu/users/varghese/PAPERS/twheel.ps.Z">'Hashed
|
||||
|
|
@ -155,7 +153,7 @@ public class HashedWheelTimer implements Timer {
|
|||
ticksPerWheel = normalizeTicksPerWheel(ticksPerWheel);
|
||||
Set<HashedWheelTimeout>[] wheel = new Set[ticksPerWheel];
|
||||
for (int i = 0; i < wheel.length; i ++) {
|
||||
wheel[i] = new MapBackedSet<HashedWheelTimeout>(new ConcurrentIdentityHashMap<HashedWheelTimeout, Boolean>(16, 0.95f, 4));
|
||||
wheel[i] = Collections.newSetFromMap(new ConcurrentIdentityHashMap<HashedWheelTimeout, Boolean>(16, 0.95f, 4));
|
||||
}
|
||||
return wheel;
|
||||
}
|
||||
|
|
@ -13,7 +13,7 @@
|
|||
* License for the specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.jboss.netty.akka.util.internal;
|
||||
package akka.util.internal;
|
||||
|
||||
import java.util.Iterator;
|
||||
|
||||
|
|
@ -13,7 +13,7 @@
|
|||
* License for the specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.jboss.netty.akka.util.internal;
|
||||
package akka.util.internal;
|
||||
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
|
|
@ -13,7 +13,7 @@
|
|||
* License for the specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.jboss.netty.akka.util;
|
||||
package akka.util.internal;
|
||||
|
||||
/**
|
||||
* A handle associated with a {@link TimerTask} that is returned by a
|
||||
|
|
@ -13,7 +13,7 @@
|
|||
* License for the specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.jboss.netty.akka.util;
|
||||
package akka.util.internal;
|
||||
|
||||
import akka.util.Duration;
|
||||
import java.util.Set;
|
||||
|
|
@ -45,7 +45,7 @@ public interface Timer {
|
|||
Timeout newTimeout(TimerTask task, Duration delay);
|
||||
|
||||
/**
|
||||
* Releases all resources acquired by this {@link org.jboss.netty.akka.util.Timer} and cancels all
|
||||
* Releases all resources acquired by this {@link Timer} and cancels all
|
||||
* tasks which were scheduled but not executed yet.
|
||||
*
|
||||
* @return the handles associated with the tasks which were canceled by
|
||||
|
|
@ -13,11 +13,11 @@
|
|||
* License for the specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.jboss.netty.akka.util;
|
||||
package akka.util.internal;
|
||||
|
||||
/**
|
||||
* A task which is executed after the delay specified with
|
||||
* {@link Timer#newTimeout(org.jboss.netty.akka.util.TimerTask, long, java.util.concurrent.TimeUnit)}
|
||||
* {@link Timer#newTimeout(TimerTask, long, java.util.concurrent.TimeUnit)}
|
||||
* .
|
||||
*
|
||||
* @author <a href="http://www.jboss.org/netty/">The Netty Project</a>
|
||||
|
|
@ -28,7 +28,7 @@ public interface TimerTask {
|
|||
|
||||
/**
|
||||
* Executed after the delay specified with
|
||||
* {@link Timer#newTimeout(org.jboss.netty.akka.util.TimerTask, long, java.util.concurrent.TimeUnit)}
|
||||
* {@link Timer#newTimeout(TimerTask, long, java.util.concurrent.TimeUnit)}
|
||||
* .
|
||||
*
|
||||
* @param timeout
|
||||
|
|
@ -1,215 +0,0 @@
|
|||
/*
|
||||
* Hex.java
|
||||
*
|
||||
* Created 04.07.2003.
|
||||
*
|
||||
* eaio: UUID - an implementation of the UUID specification Copyright (c) 2003-2009 Johann Burkard (jb@eaio.com)
|
||||
* http://eaio.com.
|
||||
*
|
||||
* Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated
|
||||
* documentation files (the "Software"), to deal in the Software without restriction, including without limitation the
|
||||
* rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to
|
||||
* permit persons to whom the Software is furnished to do so, subject to the following conditions:
|
||||
*
|
||||
* The above copyright notice and this permission notice shall be included in all copies or substantial portions of the
|
||||
* Software.
|
||||
*
|
||||
* THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE
|
||||
* WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR
|
||||
* COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR
|
||||
* OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
|
||||
*
|
||||
*/
|
||||
package com.eaio.util.lang;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Number-to-hexadecimal and hexadecimal-to-number conversions.
|
||||
*
|
||||
* @see <a href="http://johannburkard.de/software/uuid/">UUID</a>
|
||||
* @author <a href="mailto:jb@eaio.com">Johann Burkard</a>
|
||||
* @version $Id: Hex.java 1888 2009-03-15 12:43:24Z johann $
|
||||
*/
|
||||
public final class Hex {
|
||||
|
||||
/**
|
||||
* No instances needed.
|
||||
*/
|
||||
private Hex() {
|
||||
super();
|
||||
}
|
||||
|
||||
private static final char[] DIGITS = { '0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e',
|
||||
'f' };
|
||||
|
||||
/**
|
||||
* Turns a <code>short</code> into hex octets.
|
||||
*
|
||||
* @param a the {@link Appendable}, may not be <code>null</code>
|
||||
* @param in the integer
|
||||
* @return {@link Appendable}
|
||||
*/
|
||||
public static Appendable append(Appendable a, short in) {
|
||||
return append(a, (long) in, 4);
|
||||
}
|
||||
|
||||
/**
|
||||
* Turns a <code>short</code> into hex octets.
|
||||
*
|
||||
* @param a the {@link Appendable}, may not be <code>null</code>
|
||||
* @param in the integer
|
||||
* @param length the number of octets to produce
|
||||
* @return {@link Appendable}
|
||||
*/
|
||||
public static Appendable append(Appendable a, short in, int length) {
|
||||
return append(a, (long) in, length);
|
||||
}
|
||||
|
||||
/**
|
||||
* Turns an <code>int</code> into hex octets.
|
||||
*
|
||||
* @param a the {@link Appendable}, may not be <code>null</code>
|
||||
* @param in the integer
|
||||
* @return {@link Appendable}
|
||||
*/
|
||||
public static Appendable append(Appendable a, int in) {
|
||||
return append(a, (long) in, 8);
|
||||
}
|
||||
|
||||
/**
|
||||
* Turns an <code>int</code> into hex octets.
|
||||
*
|
||||
* @param a the {@link Appendable}, may not be <code>null</code>
|
||||
* @param in the integer
|
||||
* @param length the number of octets to produce
|
||||
* @return {@link Appendable}
|
||||
*/
|
||||
public static Appendable append(Appendable a, int in, int length) {
|
||||
return append(a, (long) in, length);
|
||||
}
|
||||
|
||||
/**
|
||||
* Turns a <code>long</code> into hex octets.
|
||||
*
|
||||
* @param a the {@link Appendable}, may not be <code>null</code>
|
||||
* @param in the long
|
||||
* @return {@link Appendable}
|
||||
*/
|
||||
public static Appendable append(Appendable a, long in) {
|
||||
return append(a, in, 16);
|
||||
}
|
||||
|
||||
/**
|
||||
* Turns a <code>long</code> into hex octets.
|
||||
*
|
||||
* @param a the {@link Appendable}, may not be <code>null</code>
|
||||
* @param in the long
|
||||
* @param length the number of octets to produce
|
||||
* @return {@link Appendable}
|
||||
*/
|
||||
public static Appendable append(Appendable a, long in, int length) {
|
||||
try {
|
||||
int lim = (length << 2) - 4;
|
||||
while (lim >= 0) {
|
||||
a.append(DIGITS[(byte) (in >> lim) & 0x0f]);
|
||||
lim -= 4;
|
||||
}
|
||||
}
|
||||
catch (IOException ex) {
|
||||
// Bla
|
||||
}
|
||||
return a;
|
||||
}
|
||||
|
||||
/**
|
||||
* Turns a <code>byte</code> array into hex octets.
|
||||
*
|
||||
* @param a the {@link Appendable}, may not be <code>null</code>
|
||||
* @param bytes the <code>byte</code> array
|
||||
* @return {@link Appendable}
|
||||
*/
|
||||
public static Appendable append(Appendable a, byte[] bytes) {
|
||||
try {
|
||||
for (byte b : bytes) {
|
||||
a.append(DIGITS[(byte) ((b & 0xF0) >> 4)]);
|
||||
a.append(DIGITS[(byte) (b & 0x0F)]);
|
||||
}
|
||||
}
|
||||
catch (IOException ex) {
|
||||
// Bla
|
||||
}
|
||||
return a;
|
||||
}
|
||||
|
||||
/**
|
||||
* Parses a <code>long</code> from a hex encoded number. This method will skip all characters that are not 0-9,
|
||||
* A-F and a-f.
|
||||
* <p>
|
||||
* Returns 0 if the {@link CharSequence} does not contain any interesting characters.
|
||||
*
|
||||
* @param s the {@link CharSequence} to extract a <code>long</code> from, may not be <code>null</code>
|
||||
* @return a <code>long</code>
|
||||
* @throws NullPointerException if the {@link CharSequence} is <code>null</code>
|
||||
*/
|
||||
public static long parseLong(CharSequence s) {
|
||||
long out = 0;
|
||||
byte shifts = 0;
|
||||
char c;
|
||||
for (int i = 0; i < s.length() && shifts < 16; i++) {
|
||||
c = s.charAt(i);
|
||||
if ((c > 47) && (c < 58)) {
|
||||
++shifts;
|
||||
out <<= 4;
|
||||
out |= c - 48;
|
||||
}
|
||||
else if ((c > 64) && (c < 71)) {
|
||||
++shifts;
|
||||
out <<= 4;
|
||||
out |= c - 55;
|
||||
}
|
||||
else if ((c > 96) && (c < 103)) {
|
||||
++shifts;
|
||||
out <<= 4;
|
||||
out |= c - 87;
|
||||
}
|
||||
}
|
||||
return out;
|
||||
}
|
||||
|
||||
/**
|
||||
* Parses a <code>short</code> from a hex encoded number. This method will skip all characters that are not 0-9,
|
||||
* A-F and a-f.
|
||||
* <p>
|
||||
* Returns 0 if the {@link CharSequence} does not contain any interesting characters.
|
||||
*
|
||||
* @param s the {@link CharSequence} to extract a <code>short</code> from, may not be <code>null</code>
|
||||
* @return a <code>short</code>
|
||||
* @throws NullPointerException if the {@link CharSequence} is <code>null</code>
|
||||
*/
|
||||
public static short parseShort(String s) {
|
||||
short out = 0;
|
||||
byte shifts = 0;
|
||||
char c;
|
||||
for (int i = 0; i < s.length() && shifts < 4; i++) {
|
||||
c = s.charAt(i);
|
||||
if ((c > 47) && (c < 58)) {
|
||||
++shifts;
|
||||
out <<= 4;
|
||||
out |= c - 48;
|
||||
}
|
||||
else if ((c > 64) && (c < 71)) {
|
||||
++shifts;
|
||||
out <<= 4;
|
||||
out |= c - 55;
|
||||
}
|
||||
else if ((c > 96) && (c < 103)) {
|
||||
++shifts;
|
||||
out <<= 4;
|
||||
out |= c - 87;
|
||||
}
|
||||
}
|
||||
return out;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,116 +0,0 @@
|
|||
/*
|
||||
* MACAddressParserTest.java
|
||||
*
|
||||
* Created 30.01.2006.
|
||||
*
|
||||
* eaio: UUID - an implementation of the UUID specification
|
||||
* Copyright (c) 2003-2009 Johann Burkard (jb@eaio.com) http://eaio.com.
|
||||
*
|
||||
* Permission is hereby granted, free of charge, to any person obtaining a
|
||||
* copy of this software and associated documentation files (the "Software"),
|
||||
* to deal in the Software without restriction, including without limitation
|
||||
* the rights to use, copy, modify, merge, publish, distribute, sublicense,
|
||||
* and/or sell copies of the Software, and to permit persons to whom the
|
||||
* Software is furnished to do so, subject to the following conditions:
|
||||
*
|
||||
* The above copyright notice and this permission notice shall be included
|
||||
* in all copies or substantial portions of the Software.
|
||||
*
|
||||
* THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
|
||||
* OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
|
||||
* MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN
|
||||
* NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM,
|
||||
* DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR
|
||||
* OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE
|
||||
* USE OR OTHER DEALINGS IN THE SOFTWARE.
|
||||
*
|
||||
*/
|
||||
package com.eaio.uuid;
|
||||
|
||||
/**
|
||||
* The MAC address parser attempts to find the following patterns:
|
||||
* <ul>
|
||||
* <li>.{1,2}:.{1,2}:.{1,2}:.{1,2}:.{1,2}:.{1,2}</li>
|
||||
* <li>.{1,2}-.{1,2}-.{1,2}-.{1,2}-.{1,2}-.{1,2}</li>
|
||||
* </ul>
|
||||
*
|
||||
* @see <a href="http://johannburkard.de/software/uuid/">UUID</a>
|
||||
* @author <a href="mailto:jb@eaio.com">Johann Burkard</a>
|
||||
* @version $Id: MACAddressParser.java 1888 2009-03-15 12:43:24Z johann $
|
||||
*/
|
||||
class MACAddressParser {
|
||||
|
||||
/**
|
||||
* No instances needed.
|
||||
*/
|
||||
private MACAddressParser() {
|
||||
super();
|
||||
}
|
||||
|
||||
/**
|
||||
* Attempts to find a pattern in the given String.
|
||||
*
|
||||
* @param in the String, may not be <code>null</code>
|
||||
* @return the substring that matches this pattern or <code>null</code>
|
||||
*/
|
||||
static String parse(String in) {
|
||||
|
||||
String out = in;
|
||||
|
||||
// lanscan
|
||||
|
||||
int hexStart = out.indexOf("0x");
|
||||
if (hexStart != -1 && out.indexOf("ETHER") != -1) {
|
||||
int hexEnd = out.indexOf(' ', hexStart);
|
||||
if (hexEnd > hexStart + 2) {
|
||||
out = out.substring(hexStart, hexEnd);
|
||||
}
|
||||
}
|
||||
|
||||
else {
|
||||
|
||||
int octets = 0;
|
||||
int lastIndex, old, end;
|
||||
|
||||
if (out.indexOf('-') > -1) {
|
||||
out = out.replace('-', ':');
|
||||
}
|
||||
|
||||
lastIndex = out.lastIndexOf(':');
|
||||
|
||||
if (lastIndex > out.length() - 2) {
|
||||
out = null;
|
||||
}
|
||||
else {
|
||||
|
||||
end = Math.min(out.length(), lastIndex + 3);
|
||||
|
||||
++octets;
|
||||
old = lastIndex;
|
||||
while (octets != 5 && lastIndex != -1 && lastIndex > 1) {
|
||||
lastIndex = out.lastIndexOf(':', --lastIndex);
|
||||
if (old - lastIndex == 3 || old - lastIndex == 2) {
|
||||
++octets;
|
||||
old = lastIndex;
|
||||
}
|
||||
}
|
||||
|
||||
if (octets == 5 && lastIndex > 1) {
|
||||
out = out.substring(lastIndex - 2, end).trim();
|
||||
}
|
||||
else {
|
||||
out = null;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
if (out != null && out.startsWith("0x")) {
|
||||
out = out.substring(2);
|
||||
}
|
||||
|
||||
return out;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,308 +0,0 @@
|
|||
/*
|
||||
* UUID.java
|
||||
*
|
||||
* Created 07.02.2003
|
||||
*
|
||||
* eaio: UUID - an implementation of the UUID specification
|
||||
* Copyright (c) 2003-2009 Johann Burkard (jb@eaio.com) http://eaio.com.
|
||||
*
|
||||
* Permission is hereby granted, free of charge, to any person obtaining a
|
||||
* copy of this software and associated documentation files (the "Software"),
|
||||
* to deal in the Software without restriction, including without limitation
|
||||
* the rights to use, copy, modify, merge, publish, distribute, sublicense,
|
||||
* and/or sell copies of the Software, and to permit persons to whom the
|
||||
* Software is furnished to do so, subject to the following conditions:
|
||||
*
|
||||
* The above copyright notice and this permission notice shall be included
|
||||
* in all copies or substantial portions of the Software.
|
||||
*
|
||||
* THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
|
||||
* OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
|
||||
* MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN
|
||||
* NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM,
|
||||
* DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR
|
||||
* OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE
|
||||
* USE OR OTHER DEALINGS IN THE SOFTWARE.
|
||||
*
|
||||
*/
|
||||
package com.eaio.uuid;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.ObjectInputStream;
|
||||
import java.io.ObjectOutputStream;
|
||||
import java.io.Serializable;
|
||||
|
||||
import com.eaio.util.lang.Hex;
|
||||
|
||||
/**
|
||||
* Creates UUIDs according to the DCE Universal Token Identifier specification.
|
||||
* <p>
|
||||
* All you need to know:
|
||||
* <pre>
|
||||
* UUID u = new UUID();
|
||||
* </pre>
|
||||
*
|
||||
* @see <a href="http://www.opengroup.org/onlinepubs/9629399/apdxa.htm">
|
||||
* http://www.opengroup.org/onlinepubs/9629399/apdxa.htm
|
||||
* </a>
|
||||
* @see <a href="http://www.uddi.org/pubs/draft-leach-uuids-guids-01.txt">
|
||||
* http://www.uddi.org/pubs/draft-leach-uuids-guids-01.txt
|
||||
* </a>
|
||||
* @see <a href="http://johannburkard.de/software/uuid/">UUID</a>
|
||||
* @author <a href="mailto:jb@eaio.de">Johann Burkard</a>
|
||||
* @version $Id: UUID.java 1888 2009-03-15 12:43:24Z johann $
|
||||
*/
|
||||
public class UUID implements Comparable<UUID>, Serializable, Cloneable {
|
||||
|
||||
/**
|
||||
* Hasn't ever changed between versions.
|
||||
*/
|
||||
static final long serialVersionUID = 7435962790062944603L;
|
||||
|
||||
/**
|
||||
* The time field of the UUID.
|
||||
*
|
||||
* @serial
|
||||
*/
|
||||
public long time;
|
||||
|
||||
/**
|
||||
* The clock sequence and node field of the UUID.
|
||||
*
|
||||
* @serial
|
||||
*/
|
||||
public long clockSeqAndNode;
|
||||
|
||||
/**
|
||||
* Constructor for UUID. Constructs a new, unique UUID.
|
||||
*
|
||||
* @see UUIDGen#newTime()
|
||||
* @see UUIDGen#getClockSeqAndNode()
|
||||
*/
|
||||
public UUID() {
|
||||
this(UUIDGen.newTime(), UUIDGen.getClockSeqAndNode());
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor for UUID. Constructs a UUID from two <code>long</code> values.
|
||||
*
|
||||
* @param time the upper 64 bits
|
||||
* @param clockSeqAndNode the lower 64 bits
|
||||
*/
|
||||
public UUID(long time, long clockSeqAndNode) {
|
||||
this.time = time;
|
||||
this.clockSeqAndNode = clockSeqAndNode;
|
||||
}
|
||||
|
||||
/**
|
||||
* Copy constructor for UUID. Values of the given UUID are copied.
|
||||
*
|
||||
* @param u the UUID, may not be <code>null</code>
|
||||
*/
|
||||
public UUID(UUID u) {
|
||||
this(u.time, u.clockSeqAndNode);
|
||||
}
|
||||
|
||||
/**
|
||||
* Parses a textual representation of a UUID.
|
||||
* <p>
|
||||
* No validation is performed. If the {@link CharSequence} is shorter than 36 characters,
|
||||
* {@link ArrayIndexOutOfBoundsException}s will be thrown.
|
||||
*
|
||||
* @param s the {@link CharSequence}, may not be <code>null</code>
|
||||
*/
|
||||
public UUID(CharSequence s) {
|
||||
this(Hex.parseLong(s.subSequence(0, 18)), Hex.parseLong(s.subSequence(
|
||||
19, 36)));
|
||||
}
|
||||
|
||||
/**
|
||||
* Compares this UUID to another Object. Throws a {@link ClassCastException} if
|
||||
* the other Object is not an instance of the UUID class. Returns a value
|
||||
* smaller than zero if the other UUID is "larger" than this UUID and a value
|
||||
* larger than zero if the other UUID is "smaller" than this UUID.
|
||||
*
|
||||
* @param t the other UUID, may not be <code>null</code>
|
||||
* @return a value < 0, 0 or a value > 0
|
||||
* @see java.lang.Comparable#compareTo(java.lang.Object)
|
||||
* @throws ClassCastException
|
||||
*/
|
||||
public int compareTo(UUID t) {
|
||||
if (this == t) {
|
||||
return 0;
|
||||
}
|
||||
if (time > t.time) {
|
||||
return 1;
|
||||
}
|
||||
if (time < t.time) {
|
||||
return -1;
|
||||
}
|
||||
if (clockSeqAndNode > t.clockSeqAndNode) {
|
||||
return 1;
|
||||
}
|
||||
if (clockSeqAndNode < t.clockSeqAndNode) {
|
||||
return -1;
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* Tweaked Serialization routine.
|
||||
*
|
||||
* @param out the ObjectOutputStream
|
||||
* @throws IOException
|
||||
*/
|
||||
private void writeObject(ObjectOutputStream out) throws IOException {
|
||||
out.writeLong(time);
|
||||
out.writeLong(clockSeqAndNode);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tweaked Serialization routine.
|
||||
*
|
||||
* @param in the ObjectInputStream
|
||||
* @throws IOException
|
||||
*/
|
||||
private void readObject(ObjectInputStream in) throws IOException {
|
||||
time = in.readLong();
|
||||
clockSeqAndNode = in.readLong();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns this UUID as a String.
|
||||
*
|
||||
* @return a String, never <code>null</code>
|
||||
* @see java.lang.Object#toString()
|
||||
* @see #toAppendable(Appendable)
|
||||
*/
|
||||
@Override
|
||||
public final String toString() {
|
||||
return toAppendable(null).toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Appends a String representation of this to the given {@link StringBuffer} or
|
||||
* creates a new one if none is given.
|
||||
*
|
||||
* @param in the StringBuffer to append to, may be <code>null</code>
|
||||
* @return a StringBuffer, never <code>null</code>
|
||||
* @see #toAppendable(Appendable)
|
||||
*/
|
||||
public StringBuffer toStringBuffer(StringBuffer in) {
|
||||
StringBuffer out = in;
|
||||
if (out == null) {
|
||||
out = new StringBuffer(36);
|
||||
}
|
||||
else {
|
||||
out.ensureCapacity(out.length() + 36);
|
||||
}
|
||||
return (StringBuffer) toAppendable(out);
|
||||
}
|
||||
|
||||
/**
|
||||
* Appends a String representation of this object to the given {@link Appendable} object.
|
||||
* <p>
|
||||
* For reasons I'll probably never understand, Sun has decided to have a number of I/O classes implement
|
||||
* Appendable which forced them to destroy an otherwise nice and simple interface with {@link IOException}s.
|
||||
* <p>
|
||||
* I decided to ignore any possible IOExceptions in this method.
|
||||
*
|
||||
* @param a the Appendable object, may be <code>null</code>
|
||||
* @return an Appendable object, defaults to a {@link StringBuilder} if <code>a</code> is <code>null</code>
|
||||
*/
|
||||
public Appendable toAppendable(Appendable a) {
|
||||
Appendable out = a;
|
||||
if (out == null) {
|
||||
out = new StringBuilder(36);
|
||||
}
|
||||
try {
|
||||
Hex.append(out, (int) (time >> 32)).append('-');
|
||||
Hex.append(out, (short) (time >> 16)).append('-');
|
||||
Hex.append(out, (short) time).append('-');
|
||||
Hex.append(out, (short) (clockSeqAndNode >> 48)).append('-');
|
||||
Hex.append(out, clockSeqAndNode, 12);
|
||||
}
|
||||
catch (IOException ex) {
|
||||
// What were they thinking?
|
||||
}
|
||||
return out;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a hash code of this UUID. The hash code is calculated by XOR'ing the
|
||||
* upper 32 bits of the time and clockSeqAndNode fields and the lower 32 bits of
|
||||
* the time and clockSeqAndNode fields.
|
||||
*
|
||||
* @return an <code>int</code> representing the hash code
|
||||
* @see java.lang.Object#hashCode()
|
||||
*/
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return (int) ((time >> 32) ^ time ^ (clockSeqAndNode >> 32) ^ clockSeqAndNode);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clones this UUID.
|
||||
*
|
||||
* @return a new UUID with identical values, never <code>null</code>
|
||||
*/
|
||||
@Override
|
||||
public Object clone() {
|
||||
try {
|
||||
return super.clone();
|
||||
}
|
||||
catch (CloneNotSupportedException ex) {
|
||||
// One of Sun's most epic fails.
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the time field of the UUID (upper 64 bits).
|
||||
*
|
||||
* @return the time field
|
||||
*/
|
||||
public final long getTime() {
|
||||
return time;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the clock and node field of the UUID (lower 64 bits).
|
||||
*
|
||||
* @return the clockSeqAndNode field
|
||||
*/
|
||||
public final long getClockSeqAndNode() {
|
||||
return clockSeqAndNode;
|
||||
}
|
||||
|
||||
/**
|
||||
* Compares two Objects for equality.
|
||||
*
|
||||
* @see java.lang.Object#equals(Object)
|
||||
* @param obj the Object to compare this UUID with, may be <code>null</code>
|
||||
* @return <code>true</code> if the other Object is equal to this UUID,
|
||||
* <code>false</code> if not
|
||||
*/
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (!(obj instanceof UUID)) {
|
||||
return false;
|
||||
}
|
||||
return compareTo((UUID) obj) == 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the nil UUID (a UUID whose values are both set to zero).
|
||||
* <p>
|
||||
* Starting with version 2.0, this method does return a new UUID instance every
|
||||
* time it is called. Earlier versions returned one instance. This has now been
|
||||
* changed because this UUID has public, non-final instance fields. Returning a
|
||||
* new instance is therefore more safe.
|
||||
*
|
||||
* @return a nil UUID, never <code>null</code>
|
||||
*/
|
||||
public static UUID nilUUID() {
|
||||
return new UUID(0, 0);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,368 +0,0 @@
|
|||
/*
|
||||
* UUIDGen.java
|
||||
*
|
||||
* Created on 09.08.2003.
|
||||
*
|
||||
* eaio: UUID - an implementation of the UUID specification
|
||||
* Copyright (c) 2003-2009 Johann Burkard (jb@eaio.com) http://eaio.com.
|
||||
*
|
||||
* Permission is hereby granted, free of charge, to any person obtaining a
|
||||
* copy of this software and associated documentation files (the "Software"),
|
||||
* to deal in the Software without restriction, including without limitation
|
||||
* the rights to use, copy, modify, merge, publish, distribute, sublicense,
|
||||
* and/or sell copies of the Software, and to permit persons to whom the
|
||||
* Software is furnished to do so, subject to the following conditions:
|
||||
*
|
||||
* The above copyright notice and this permission notice shall be included
|
||||
* in all copies or substantial portions of the Software.
|
||||
*
|
||||
* THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
|
||||
* OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
|
||||
* MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN
|
||||
* NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM,
|
||||
* DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR
|
||||
* OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE
|
||||
* USE OR OTHER DEALINGS IN THE SOFTWARE.
|
||||
*
|
||||
*/
|
||||
package com.eaio.uuid;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStreamReader;
|
||||
import java.net.InetAddress;
|
||||
import java.net.InterfaceAddress;
|
||||
import java.net.NetworkInterface;
|
||||
import java.net.SocketException;
|
||||
import java.net.UnknownHostException;
|
||||
import java.util.Enumeration;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import com.eaio.util.lang.Hex;
|
||||
|
||||
/**
|
||||
* This class contains methods to generate UUID fields. These methods have been
|
||||
* refactored out of {@link com.eaio.uuid.UUID}.
|
||||
* <p>
|
||||
* Starting with version 2, this implementation tries to obtain the MAC address
|
||||
* of the network card. Under Microsoft Windows, the <code>ifconfig</code>
|
||||
* command is used which may pop up a command window in Java Virtual Machines
|
||||
* prior to 1.4 once this class is initialized. The command window is closed
|
||||
* automatically.
|
||||
* <p>
|
||||
* The MAC address code has been tested extensively in Microsoft Windows,
|
||||
* Linux, Solaris 8, HP-UX 11, but should work in MacOS X and BSDs, too.
|
||||
* <p>
|
||||
* If you use JDK 6 or later, the code in {@link InterfaceAddress} will be used.
|
||||
*
|
||||
* @see <a href="http://johannburkard.de/software/uuid/">UUID</a>
|
||||
* @author <a href="mailto:jb@eaio.de">Johann Burkard</a>
|
||||
* @version $Id: UUIDGen.java 2914 2010-04-23 11:35:00Z johann $
|
||||
* @see com.eaio.uuid.UUID
|
||||
*/
|
||||
public final class UUIDGen {
|
||||
|
||||
/**
|
||||
* No instances needed.
|
||||
*/
|
||||
private UUIDGen() {
|
||||
super();
|
||||
}
|
||||
|
||||
/**
|
||||
* The last time value. Used to remove duplicate UUIDs.
|
||||
*/
|
||||
private final static AtomicLong lastTime = new AtomicLong(Long.MIN_VALUE);
|
||||
|
||||
/**
|
||||
* The cached MAC address.
|
||||
*/
|
||||
private static String macAddress = null;
|
||||
|
||||
/**
|
||||
* The current clock and node value.
|
||||
*/
|
||||
private static long clockSeqAndNode = 0x8000000000000000L;
|
||||
|
||||
static {
|
||||
|
||||
try {
|
||||
Class.forName("java.net.InterfaceAddress");
|
||||
macAddress = Class.forName(
|
||||
"com.eaio.uuid.UUIDGen$HardwareAddressLookup").newInstance().toString();
|
||||
}
|
||||
catch (ExceptionInInitializerError err) {
|
||||
// Ignored.
|
||||
}
|
||||
catch (ClassNotFoundException ex) {
|
||||
// Ignored.
|
||||
}
|
||||
catch (LinkageError err) {
|
||||
// Ignored.
|
||||
}
|
||||
catch (IllegalAccessException ex) {
|
||||
// Ignored.
|
||||
}
|
||||
catch (InstantiationException ex) {
|
||||
// Ignored.
|
||||
}
|
||||
catch (SecurityException ex) {
|
||||
// Ignored.
|
||||
}
|
||||
|
||||
if (macAddress == null) {
|
||||
|
||||
Process p = null;
|
||||
BufferedReader in = null;
|
||||
|
||||
try {
|
||||
String osname = System.getProperty("os.name", "");
|
||||
|
||||
if (osname.startsWith("Windows")) {
|
||||
p = Runtime.getRuntime().exec(
|
||||
new String[] { "ipconfig", "/all" }, null);
|
||||
}
|
||||
// Solaris code must appear before the generic code
|
||||
else if (osname.startsWith("Solaris")
|
||||
|| osname.startsWith("SunOS")) {
|
||||
String hostName = getFirstLineOfCommand(
|
||||
"uname", "-n" );
|
||||
if (hostName != null) {
|
||||
p = Runtime.getRuntime().exec(
|
||||
new String[] { "/usr/sbin/arp", hostName },
|
||||
null);
|
||||
}
|
||||
}
|
||||
else if (new File("/usr/sbin/lanscan").exists()) {
|
||||
p = Runtime.getRuntime().exec(
|
||||
new String[] { "/usr/sbin/lanscan" }, null);
|
||||
}
|
||||
else if (new File("/sbin/ifconfig").exists()) {
|
||||
p = Runtime.getRuntime().exec(
|
||||
new String[] { "/sbin/ifconfig", "-a" }, null);
|
||||
}
|
||||
|
||||
if (p != null) {
|
||||
in = new BufferedReader(new InputStreamReader(
|
||||
p.getInputStream()), 128);
|
||||
String l = null;
|
||||
while ((l = in.readLine()) != null) {
|
||||
macAddress = MACAddressParser.parse(l);
|
||||
if (macAddress != null
|
||||
&& Hex.parseShort(macAddress) != 0xff) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
catch (SecurityException ex) {
|
||||
// Ignore it.
|
||||
}
|
||||
catch (IOException ex) {
|
||||
// Ignore it.
|
||||
}
|
||||
finally {
|
||||
if (p != null) {
|
||||
if (in != null) {
|
||||
try {
|
||||
in.close();
|
||||
}
|
||||
catch (IOException ex) {
|
||||
// Ignore it.
|
||||
}
|
||||
}
|
||||
try {
|
||||
p.getErrorStream().close();
|
||||
}
|
||||
catch (IOException ex) {
|
||||
// Ignore it.
|
||||
}
|
||||
try {
|
||||
p.getOutputStream().close();
|
||||
}
|
||||
catch (IOException ex) {
|
||||
// Ignore it.
|
||||
}
|
||||
p.destroy();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
if (macAddress != null) {
|
||||
clockSeqAndNode |= Hex.parseLong(macAddress);
|
||||
}
|
||||
else {
|
||||
try {
|
||||
byte[] local = InetAddress.getLocalHost().getAddress();
|
||||
clockSeqAndNode |= (local[0] << 24) & 0xFF000000L;
|
||||
clockSeqAndNode |= (local[1] << 16) & 0xFF0000;
|
||||
clockSeqAndNode |= (local[2] << 8) & 0xFF00;
|
||||
clockSeqAndNode |= local[3] & 0xFF;
|
||||
}
|
||||
catch (UnknownHostException ex) {
|
||||
clockSeqAndNode |= (long) (Math.random() * 0x7FFFFFFF);
|
||||
}
|
||||
}
|
||||
|
||||
// Skip the clock sequence generation process and use random instead.
|
||||
|
||||
clockSeqAndNode |= (long) (Math.random() * 0x3FFF) << 48;
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the current clockSeqAndNode value.
|
||||
*
|
||||
* @return the clockSeqAndNode value
|
||||
* @see UUID#getClockSeqAndNode()
|
||||
*/
|
||||
public static long getClockSeqAndNode() {
|
||||
return clockSeqAndNode;
|
||||
}
|
||||
|
||||
/**
|
||||
* Generates a new time field. Each time field is unique and larger than the
|
||||
* previously generated time field.
|
||||
*
|
||||
* @return a new time value
|
||||
* @see UUID#getTime()
|
||||
*/
|
||||
public static long newTime() {
|
||||
return createTime(System.currentTimeMillis());
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new time field from the given timestamp. Note that even identical
|
||||
* values of <code>currentTimeMillis</code> will produce different time fields.
|
||||
*
|
||||
* @param currentTimeMillis the timestamp
|
||||
* @return a new time value
|
||||
* @see UUID#getTime()
|
||||
*/
|
||||
public static long createTime(long currentTimeMillis) {
|
||||
|
||||
long time;
|
||||
|
||||
// UTC time
|
||||
|
||||
long timeMillis = (currentTimeMillis * 10000) + 0x01B21DD213814000L;
|
||||
|
||||
// Make sure our time is unique
|
||||
|
||||
for(;;) {
|
||||
final long c = lastTime.get();
|
||||
if (timeMillis <= c) {
|
||||
timeMillis = lastTime.incrementAndGet();
|
||||
break;
|
||||
} else if(lastTime.compareAndSet(c, timeMillis)) break;
|
||||
}
|
||||
|
||||
// time low
|
||||
|
||||
time = timeMillis << 32;
|
||||
|
||||
// time mid
|
||||
|
||||
time |= (timeMillis & 0xFFFF00000000L) >> 16;
|
||||
|
||||
// time hi and version
|
||||
|
||||
time |= 0x1000 | ((timeMillis >> 48) & 0x0FFF); // version 1
|
||||
|
||||
return time;
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the MAC address. Not guaranteed to return anything.
|
||||
*
|
||||
* @return the MAC address, may be <code>null</code>
|
||||
*/
|
||||
public static String getMACAddress() {
|
||||
return macAddress;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the first line of the shell command.
|
||||
*
|
||||
* @param commands the commands to run
|
||||
* @return the first line of the command
|
||||
* @throws IOException
|
||||
*/
|
||||
static String getFirstLineOfCommand(String... commands) throws IOException {
|
||||
|
||||
Process p = null;
|
||||
BufferedReader reader = null;
|
||||
|
||||
try {
|
||||
p = Runtime.getRuntime().exec(commands);
|
||||
reader = new BufferedReader(new InputStreamReader(
|
||||
p.getInputStream()), 128);
|
||||
|
||||
return reader.readLine();
|
||||
}
|
||||
finally {
|
||||
if (p != null) {
|
||||
if (reader != null) {
|
||||
try {
|
||||
reader.close();
|
||||
}
|
||||
catch (IOException ex) {
|
||||
// Ignore it.
|
||||
}
|
||||
}
|
||||
try {
|
||||
p.getErrorStream().close();
|
||||
}
|
||||
catch (IOException ex) {
|
||||
// Ignore it.
|
||||
}
|
||||
try {
|
||||
p.getOutputStream().close();
|
||||
}
|
||||
catch (IOException ex) {
|
||||
// Ignore it.
|
||||
}
|
||||
p.destroy();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Scans MAC addresses for good ones.
|
||||
*/
|
||||
static class HardwareAddressLookup {
|
||||
|
||||
/**
|
||||
* @see java.lang.Object#toString()
|
||||
*/
|
||||
@Override
|
||||
public String toString() {
|
||||
String out = null;
|
||||
try {
|
||||
Enumeration<NetworkInterface> ifs = NetworkInterface.getNetworkInterfaces();
|
||||
if (ifs != null) {
|
||||
while (ifs.hasMoreElements()) {
|
||||
NetworkInterface iface = ifs.nextElement();
|
||||
byte[] hardware = iface.getHardwareAddress();
|
||||
if (hardware != null && hardware.length == 6
|
||||
&& hardware[1] != (byte) 0xff) {
|
||||
out = Hex.append(new StringBuilder(36), hardware).toString();
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (SocketException ex) {
|
||||
// Ignore it.
|
||||
}
|
||||
return out;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,73 +0,0 @@
|
|||
/*
|
||||
* Copyright 2009 Red Hat, Inc.
|
||||
*
|
||||
* Red Hat licenses this file to you under the Apache License, version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with the
|
||||
* License. You may obtain a copy of the License at:
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
* License for the specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
package org.jboss.netty.akka.util;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.AbstractSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* A {@link java.util.Map}-backed {@link java.util.Set}.
|
||||
*
|
||||
* @author <a href="http://www.jboss.org/netty/">The Netty Project</a>
|
||||
* @author <a href="http://gleamynode.net/">Trustin Lee</a>
|
||||
*
|
||||
* @version $Rev: 2080 $, $Date: 2010-01-26 18:04:19 +0900 (Tue, 26 Jan 2010) $
|
||||
*/
|
||||
final class MapBackedSet<E> extends AbstractSet<E> implements Serializable {
|
||||
|
||||
private static final long serialVersionUID = -6761513279741915432L;
|
||||
|
||||
private final Map<E, Boolean> map;
|
||||
|
||||
/**
|
||||
* Creates a new instance which wraps the specified {@code map}.
|
||||
*/
|
||||
MapBackedSet(Map<E, Boolean> map) {
|
||||
this.map = map;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int size() {
|
||||
return map.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean contains(Object o) {
|
||||
return map.containsKey(o);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean add(E o) {
|
||||
return map.put(o, Boolean.TRUE) == null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean remove(Object o) {
|
||||
return map.remove(o) != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clear() {
|
||||
map.clear();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<E> iterator() {
|
||||
return map.keySet().iterator();
|
||||
}
|
||||
}
|
||||
|
|
@ -4,26 +4,6 @@
|
|||
|
||||
package akka
|
||||
|
||||
import akka.actor.newUuid
|
||||
|
||||
object AkkaException {
|
||||
|
||||
def toStringWithStackTrace(throwable: Throwable): String = throwable match {
|
||||
case null ⇒ "Unknown Throwable: was 'null'"
|
||||
case ae: AkkaException ⇒ ae.toLongString
|
||||
case e ⇒ "%s:%s\n%s" format (e.getClass.getName, e.getMessage, stackTraceToString(e))
|
||||
}
|
||||
|
||||
def stackTraceToString(throwable: Throwable): String = {
|
||||
val trace = throwable.getStackTrace
|
||||
val sb = new StringBuilder
|
||||
for (i ← 0 until trace.length)
|
||||
sb.append("\tat %s\n" format trace(i))
|
||||
sb.toString
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Akka base Exception. Each Exception gets:
|
||||
* <ul>
|
||||
|
|
@ -33,16 +13,17 @@ object AkkaException {
|
|||
* </ul>
|
||||
*/
|
||||
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
|
||||
class AkkaException(message: String = "", cause: Throwable = null) extends RuntimeException(message, cause) with Serializable {
|
||||
lazy val uuid = newUuid.toString
|
||||
class AkkaException(message: String, cause: Throwable) extends RuntimeException(message, cause) with Serializable {
|
||||
def this(msg: String) = this(msg, null)
|
||||
|
||||
override lazy val toString =
|
||||
"%s:%s\n[%s]".format(getClass.getName, message, uuid)
|
||||
lazy val uuid: String = java.util.UUID.randomUUID().toString
|
||||
|
||||
lazy val toLongString =
|
||||
"%s:%s\n[%s]\n%s".format(getClass.getName, message, uuid, stackTraceToString)
|
||||
|
||||
def this(msg: String) = this(msg, null);
|
||||
|
||||
def stackTraceToString = AkkaException.stackTraceToString(this)
|
||||
override def toString(): String = uuid + super.toString()
|
||||
}
|
||||
|
||||
/**
|
||||
* This exception is thrown when Akka detects a problem with the provided configuration
|
||||
*/
|
||||
class ConfigurationException(message: String, cause: Throwable) extends AkkaException(message, cause) {
|
||||
def this(msg: String) = this(msg, null)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -12,8 +12,9 @@ import java.util.regex.Pattern
|
|||
|
||||
/**
|
||||
* Marker trait to show which Messages are automatically handled by Akka
|
||||
* Internal use only
|
||||
*/
|
||||
trait AutoReceivedMessage extends Serializable
|
||||
private[akka] trait AutoReceivedMessage extends Serializable
|
||||
|
||||
/**
|
||||
* Marker trait to indicate that a message might be potentially harmful,
|
||||
|
|
@ -26,9 +27,16 @@ trait PossiblyHarmful
|
|||
*/
|
||||
trait NoSerializationVerificationNeeded
|
||||
|
||||
case class Failed(cause: Throwable) extends AutoReceivedMessage with PossiblyHarmful
|
||||
/**
|
||||
* Internal use only
|
||||
*/
|
||||
private[akka] case class Failed(cause: Throwable) extends AutoReceivedMessage with PossiblyHarmful
|
||||
|
||||
abstract class PoisonPill extends AutoReceivedMessage with PossiblyHarmful
|
||||
|
||||
/**
|
||||
* A message all Actors will understand, that when processed will terminate the Actor permanently.
|
||||
*/
|
||||
case object PoisonPill extends PoisonPill {
|
||||
/**
|
||||
* Java API: get the singleton instance
|
||||
|
|
@ -37,6 +45,10 @@ case object PoisonPill extends PoisonPill {
|
|||
}
|
||||
|
||||
abstract class Kill extends AutoReceivedMessage with PossiblyHarmful
|
||||
/**
|
||||
* A message all Actors will understand, that when processed will make the Actor throw an ActorKilledException,
|
||||
* which will trigger supervision.
|
||||
*/
|
||||
case object Kill extends Kill {
|
||||
/**
|
||||
* Java API: get the singleton instance
|
||||
|
|
@ -44,9 +56,17 @@ case object Kill extends Kill {
|
|||
def getInstance = this
|
||||
}
|
||||
|
||||
/**
|
||||
* When Death Watch is used, the watcher will receive a Terminated(watched) message when watched is terminated.
|
||||
*/
|
||||
case class Terminated(@BeanProperty actor: ActorRef) extends PossiblyHarmful
|
||||
|
||||
abstract class ReceiveTimeout extends PossiblyHarmful
|
||||
|
||||
/**
|
||||
* When using ActorContext.setReceiveTimeout, the singleton instance of ReceiveTimeout will be sent
|
||||
* to the Actor when there hasn't been any message for that long.
|
||||
*/
|
||||
case object ReceiveTimeout extends ReceiveTimeout {
|
||||
/**
|
||||
* Java API: get the singleton instance
|
||||
|
|
@ -60,49 +80,79 @@ case object ReceiveTimeout extends ReceiveTimeout {
|
|||
* message is delivered by active routing of the various actors involved.
|
||||
*/
|
||||
sealed trait SelectionPath extends AutoReceivedMessage
|
||||
case class SelectChildName(name: String, next: Any) extends SelectionPath
|
||||
case class SelectChildPattern(pattern: Pattern, next: Any) extends SelectionPath
|
||||
case class SelectParent(next: Any) extends SelectionPath
|
||||
|
||||
// Exceptions for Actors
|
||||
/**
|
||||
* Internal use only
|
||||
*/
|
||||
private[akka] case class SelectChildName(name: String, next: Any) extends SelectionPath
|
||||
|
||||
/**
|
||||
* Internal use only
|
||||
*/
|
||||
private[akka] case class SelectChildPattern(pattern: Pattern, next: Any) extends SelectionPath
|
||||
|
||||
/**
|
||||
* Internal use only
|
||||
*/
|
||||
private[akka] case class SelectParent(next: Any) extends SelectionPath
|
||||
|
||||
/**
|
||||
* IllegalActorStateException is thrown when a core invariant in the Actor implementation has been violated.
|
||||
* For instance, if you try to create an Actor that doesn't extend Actor.
|
||||
*/
|
||||
class IllegalActorStateException private[akka] (message: String, cause: Throwable = null)
|
||||
extends AkkaException(message, cause) {
|
||||
def this(msg: String) = this(msg, null);
|
||||
def this(msg: String) = this(msg, null)
|
||||
}
|
||||
|
||||
/**
|
||||
* ActorKilledException is thrown when an Actor receives the akka.actor.Kill message
|
||||
*/
|
||||
class ActorKilledException private[akka] (message: String, cause: Throwable)
|
||||
extends AkkaException(message, cause)
|
||||
with NoStackTrace {
|
||||
def this(msg: String) = this(msg, null);
|
||||
def this(msg: String) = this(msg, null)
|
||||
}
|
||||
|
||||
case class InvalidActorNameException(message: String) extends AkkaException(message)
|
||||
/**
|
||||
* An InvalidActorNameException is thrown when you try to convert something, usually a String, to an Actor name
|
||||
* which doesn't validate.
|
||||
*/
|
||||
class InvalidActorNameException(message: String) extends AkkaException(message)
|
||||
|
||||
case class ActorInitializationException private[akka] (actor: ActorRef, message: String, cause: Throwable = null)
|
||||
extends AkkaException(message, cause)
|
||||
with NoStackTrace {
|
||||
def this(msg: String) = this(null, msg, null);
|
||||
}
|
||||
|
||||
class ActorTimeoutException private[akka] (message: String, cause: Throwable = null)
|
||||
extends AkkaException(message, cause) {
|
||||
def this(msg: String) = this(msg, null);
|
||||
/**
|
||||
* An ActorInitializationException is thrown when the the initialization logic for an Actor fails.
|
||||
*/
|
||||
class ActorInitializationException private[akka] (actor: ActorRef, message: String, cause: Throwable)
|
||||
extends AkkaException(message, cause) /*with NoStackTrace*/ {
|
||||
def this(msg: String) = this(null, msg, null)
|
||||
def this(actor: ActorRef, msg: String) = this(actor, msg, null)
|
||||
}
|
||||
|
||||
/**
|
||||
* InvalidMessageException is thrown when an invalid message is sent to an Actor.
|
||||
* Technically it's only "null" which is an InvalidMessageException but who knows,
|
||||
* there might be more of them in the future, or not.
|
||||
*/
|
||||
class InvalidMessageException private[akka] (message: String, cause: Throwable = null)
|
||||
extends AkkaException(message, cause)
|
||||
with NoStackTrace {
|
||||
def this(msg: String) = this(msg, null);
|
||||
def this(msg: String) = this(msg, null)
|
||||
}
|
||||
|
||||
/**
|
||||
* A DeathPactException is thrown by an Actor that receives a Terminated(someActor) message
|
||||
* that it doesn't handle itself, effectively crashing the Actor and escalating to the supervisor.
|
||||
*/
|
||||
case class DeathPactException private[akka] (dead: ActorRef)
|
||||
extends AkkaException("Monitored actor [" + dead + "] terminated")
|
||||
with NoStackTrace
|
||||
|
||||
// must not pass InterruptedException to other threads
|
||||
case class ActorInterruptedException private[akka] (cause: Throwable)
|
||||
extends AkkaException(cause.getMessage, cause)
|
||||
with NoStackTrace
|
||||
/**
|
||||
* When an InterruptedException is thrown inside an Actor, it is wrapped as an ActorInterruptedException as to
|
||||
* avoid cascading interrupts to other threads than the originally interrupted one.
|
||||
*/
|
||||
class ActorInterruptedException private[akka] (cause: Throwable) extends AkkaException(cause.getMessage, cause) with NoStackTrace
|
||||
|
||||
/**
|
||||
* This message is published to the EventStream whenever an Actor receives a message it doesn't understand
|
||||
|
|
@ -115,18 +165,43 @@ case class UnhandledMessage(@BeanProperty message: Any, @BeanProperty sender: Ac
|
|||
*/
|
||||
object Status {
|
||||
sealed trait Status extends Serializable
|
||||
|
||||
/**
|
||||
* This class/message type is preferably used to indicate success of some operation performed.
|
||||
*/
|
||||
case class Success(status: AnyRef) extends Status
|
||||
|
||||
/**
|
||||
* This class/message type is preferably used to indicate failure of some operation performed.
|
||||
* As an example, it is used to signal failure with AskSupport is used (ask/?).
|
||||
*/
|
||||
case class Failure(cause: Throwable) extends Status
|
||||
}
|
||||
|
||||
/**
|
||||
* Mix in ActorLogging into your Actor to easily obtain a reference to a logger, which is available under the name "log".
|
||||
*
|
||||
* {{{
|
||||
* class MyActor extends Actor with ActorLogging {
|
||||
* def receive = {
|
||||
* case "pigdog" => log.info("We've got yet another pigdog on our hands")
|
||||
* }
|
||||
* }
|
||||
* }}}
|
||||
*/
|
||||
trait ActorLogging { this: Actor ⇒
|
||||
val log = akka.event.Logging(context.system, this)
|
||||
}
|
||||
|
||||
object Actor {
|
||||
|
||||
/**
|
||||
* Type alias representing a Receive-expression for Akka Actors.
|
||||
*/
|
||||
type Receive = PartialFunction[Any, Unit]
|
||||
|
||||
/**
|
||||
* emptyBehavior is a Receive-expression that matches no messages at all, ever.
|
||||
*/
|
||||
object emptyBehavior extends Receive {
|
||||
def isDefinedAt(x: Any) = false
|
||||
def apply(x: Any) = throw new UnsupportedOperationException("Empty behavior apply()")
|
||||
|
|
@ -243,7 +318,7 @@ trait Actor {
|
|||
* This defines the initial actor behavior, it must return a partial function
|
||||
* with the actor logic.
|
||||
*/
|
||||
protected def receive: Receive
|
||||
def receive: Receive
|
||||
|
||||
/**
|
||||
* User overridable definition the strategy to use for supervising
|
||||
|
|
@ -303,42 +378,5 @@ trait Actor {
|
|||
case _ ⇒ context.system.eventStream.publish(UnhandledMessage(message, sender, self))
|
||||
}
|
||||
}
|
||||
|
||||
// =========================================
|
||||
// ==== INTERNAL IMPLEMENTATION DETAILS ====
|
||||
// =========================================
|
||||
|
||||
/**
|
||||
* For Akka internal use only.
|
||||
*/
|
||||
private[akka] final def apply(msg: Any) = {
|
||||
// TODO would it be more efficient to assume that most messages are matched and catch MatchError instead of using isDefinedAt?
|
||||
val head = behaviorStack.head
|
||||
if (head.isDefinedAt(msg)) head.apply(msg) else unhandled(msg)
|
||||
}
|
||||
|
||||
/**
|
||||
* For Akka internal use only.
|
||||
*/
|
||||
private[akka] def pushBehavior(behavior: Receive): Unit = {
|
||||
behaviorStack = behaviorStack.push(behavior)
|
||||
}
|
||||
|
||||
/**
|
||||
* For Akka internal use only.
|
||||
*/
|
||||
private[akka] def popBehavior(): Unit = {
|
||||
val original = behaviorStack
|
||||
val popped = original.pop
|
||||
behaviorStack = if (popped.isEmpty) original else popped
|
||||
}
|
||||
|
||||
/**
|
||||
* For Akka internal use only.
|
||||
*/
|
||||
private[akka] def clearBehaviorStack(): Unit =
|
||||
behaviorStack = Stack.empty[Receive].push(behaviorStack.last)
|
||||
|
||||
private var behaviorStack: Stack[Receive] = Stack.empty[Receive].push(receive)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -134,10 +134,17 @@ trait ActorContext extends ActorRefFactory {
|
|||
*/
|
||||
def unwatch(subject: ActorRef): ActorRef
|
||||
|
||||
/**
|
||||
* ActorContexts shouldn't be Serializable
|
||||
*/
|
||||
final protected def writeObject(o: ObjectOutputStream): Unit =
|
||||
throw new NotSerializableException("ActorContext is not serializable!")
|
||||
}
|
||||
|
||||
/**
|
||||
* UntypedActorContext is the UntypedActor equivalent of ActorContext,
|
||||
* containing the Java API
|
||||
*/
|
||||
trait UntypedActorContext extends ActorContext {
|
||||
|
||||
/**
|
||||
|
|
@ -178,7 +185,9 @@ private[akka] object ActorCell {
|
|||
|
||||
final val emptyReceiveTimeoutData: (Long, Cancellable) = (-1, emptyCancellable)
|
||||
|
||||
trait SuspendReason
|
||||
final val behaviorStackPlaceHolder: Stack[Actor.Receive] = Stack.empty.push(Actor.emptyBehavior)
|
||||
|
||||
sealed trait SuspendReason
|
||||
case object UserRequest extends SuspendReason
|
||||
case class Recreation(cause: Throwable) extends SuspendReason
|
||||
case object Termination extends SuspendReason
|
||||
|
|
@ -402,6 +411,8 @@ private[akka] class ActorCell(
|
|||
|
||||
var actor: Actor = _
|
||||
|
||||
private var behaviorStack: Stack[Actor.Receive] = Stack.empty
|
||||
|
||||
@volatile //This must be volatile since it isn't protected by the mailbox status
|
||||
var mailbox: Mailbox = _
|
||||
|
||||
|
|
@ -482,14 +493,20 @@ private[akka] class ActorCell(
|
|||
|
||||
//This method is in charge of setting up the contextStack and create a new instance of the Actor
|
||||
protected def newActor(): Actor = {
|
||||
val stackBefore = contextStack.get
|
||||
contextStack.set(stackBefore.push(this))
|
||||
contextStack.set(contextStack.get.push(this))
|
||||
try {
|
||||
val instance = props.creator()
|
||||
import ActorCell.behaviorStackPlaceHolder
|
||||
|
||||
behaviorStack = behaviorStackPlaceHolder
|
||||
val instance = props.creator.apply()
|
||||
|
||||
if (instance eq null)
|
||||
throw ActorInitializationException(self, "Actor instance passed to actorOf can't be 'null'")
|
||||
throw new ActorInitializationException(self, "Actor instance passed to actorOf can't be 'null'")
|
||||
|
||||
behaviorStack = behaviorStack match {
|
||||
case `behaviorStackPlaceHolder` ⇒ Stack.empty.push(instance.receive)
|
||||
case newBehaviors ⇒ Stack.empty.push(instance.receive).pushAll(newBehaviors.reverse.drop(1))
|
||||
}
|
||||
instance
|
||||
} finally {
|
||||
val stackAfter = contextStack.get
|
||||
|
|
@ -510,13 +527,12 @@ private[akka] class ActorCell(
|
|||
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(created), "started (" + created + ")"))
|
||||
} catch {
|
||||
case NonFatal(i: InstantiationException) ⇒
|
||||
throw ActorInitializationException(self,
|
||||
throw new ActorInitializationException(self,
|
||||
"""exception during creation, this problem is likely to occur because the class of the Actor you tried to create is either,
|
||||
a non-static inner class (in which case make it a static inner class or use Props(new ...) or Props( new UntypedActorFactory ... )
|
||||
or is missing an appropriate, reachable no-args constructor.
|
||||
""", i.getCause)
|
||||
case NonFatal(e) ⇒
|
||||
throw ActorInitializationException(self, "exception during creation", e)
|
||||
case NonFatal(e) ⇒ throw new ActorInitializationException(self, "exception during creation", e)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -540,7 +556,10 @@ private[akka] class ActorCell(
|
|||
doRecreate(cause, failedActor)
|
||||
}
|
||||
} catch {
|
||||
case NonFatal(e) ⇒ throw ActorInitializationException(self, "exception during creation", e)
|
||||
case NonFatal(e) ⇒ throw new ActorInitializationException(self, "exception during creation", e match {
|
||||
case i: InstantiationException ⇒ i.getCause
|
||||
case other ⇒ other
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -605,7 +624,7 @@ private[akka] class ActorCell(
|
|||
cancelReceiveTimeout() // FIXME: leave this here???
|
||||
messageHandle.message match {
|
||||
case msg: AutoReceivedMessage ⇒ autoReceiveMessage(messageHandle)
|
||||
case msg ⇒ actor(msg)
|
||||
case msg ⇒ receiveMessage(msg)
|
||||
}
|
||||
currentMessage = null // reset current message after successful invocation
|
||||
} catch {
|
||||
|
|
@ -621,14 +640,14 @@ private[akka] class ActorCell(
|
|||
if (actor ne null) actor.supervisorStrategy.handleSupervisorFailing(self, children)
|
||||
} finally {
|
||||
t match { // Wrap InterruptedExceptions and rethrow
|
||||
case _: InterruptedException ⇒ parent.tell(Failed(ActorInterruptedException(t)), self); throw t
|
||||
case _: InterruptedException ⇒ parent.tell(Failed(new ActorInterruptedException(t)), self); throw t
|
||||
case _ ⇒ parent.tell(Failed(t), self)
|
||||
}
|
||||
}
|
||||
|
||||
def become(behavior: Actor.Receive, discardOld: Boolean = true): Unit = {
|
||||
if (discardOld) unbecome()
|
||||
actor.pushBehavior(behavior)
|
||||
behaviorStack = behaviorStack.push(behavior)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -639,14 +658,16 @@ private[akka] class ActorCell(
|
|||
/*
|
||||
* UntypedActorContext impl
|
||||
*/
|
||||
def become(behavior: Procedure[Any], discardOld: Boolean): Unit = {
|
||||
def newReceive: Actor.Receive = { case msg ⇒ behavior.apply(msg) }
|
||||
become(newReceive, discardOld)
|
||||
def become(behavior: Procedure[Any], discardOld: Boolean): Unit =
|
||||
become({ case msg ⇒ behavior.apply(msg) }: Actor.Receive, discardOld)
|
||||
|
||||
def unbecome(): Unit = {
|
||||
val original = behaviorStack
|
||||
val popped = original.pop
|
||||
behaviorStack = if (popped.isEmpty) original else popped
|
||||
}
|
||||
|
||||
def unbecome(): Unit = actor.popBehavior()
|
||||
|
||||
def autoReceiveMessage(msg: Envelope) {
|
||||
def autoReceiveMessage(msg: Envelope): Unit = {
|
||||
if (system.settings.DebugAutoReceive)
|
||||
system.eventStream.publish(Debug(self.path.toString, clazz(actor), "received AutoReceiveMessage " + msg))
|
||||
|
||||
|
|
@ -660,6 +681,12 @@ private[akka] class ActorCell(
|
|||
}
|
||||
}
|
||||
|
||||
final def receiveMessage(msg: Any): Unit = {
|
||||
//FIXME replace with behaviorStack.head.applyOrElse(msg, unhandled) + "-optimize"
|
||||
val head = behaviorStack.head
|
||||
if (head.isDefinedAt(msg)) head.apply(msg) else actor.unhandled(msg)
|
||||
}
|
||||
|
||||
private def doTerminate() {
|
||||
val a = actor
|
||||
try {
|
||||
|
|
@ -675,7 +702,7 @@ private[akka] class ActorCell(
|
|||
if (system.settings.DebugLifecycle)
|
||||
system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopped"))
|
||||
} finally {
|
||||
if (a ne null) a.clearBehaviorStack()
|
||||
behaviorStack = ActorCell.behaviorStackPlaceHolder
|
||||
clearActorFields(a)
|
||||
actor = null
|
||||
}
|
||||
|
|
@ -685,7 +712,6 @@ private[akka] class ActorCell(
|
|||
private def doRecreate(cause: Throwable, failedActor: Actor): Unit = try {
|
||||
// after all killed children have terminated, recreate the rest, then go on to start the new instance
|
||||
actor.supervisorStrategy.handleSupervisorRestarted(cause, self, children)
|
||||
|
||||
val freshActor = newActor()
|
||||
actor = freshActor // this must happen before postRestart has a chance to fail
|
||||
if (freshActor eq failedActor) setActorFields(freshActor, this, self) // If the creator returns the same instance, we need to restore our nulled out fields.
|
||||
|
|
@ -702,7 +728,7 @@ private[akka] class ActorCell(
|
|||
actor.supervisorStrategy.handleSupervisorFailing(self, children)
|
||||
clearActorFields(actor) // If this fails, we need to ensure that preRestart isn't called.
|
||||
} finally {
|
||||
parent.tell(Failed(ActorInitializationException(self, "exception during re-creation", e)), self)
|
||||
parent.tell(Failed(new ActorInitializationException(self, "exception during re-creation", e)), self)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -749,13 +775,11 @@ private[akka] class ActorCell(
|
|||
|
||||
}
|
||||
|
||||
final def cancelReceiveTimeout() {
|
||||
//Only cancel if
|
||||
final def cancelReceiveTimeout(): Unit =
|
||||
if (receiveTimeoutData._2 ne emptyCancellable) {
|
||||
receiveTimeoutData._2.cancel()
|
||||
receiveTimeoutData = (receiveTimeoutData._1, emptyCancellable)
|
||||
}
|
||||
}
|
||||
|
||||
final def clearActorFields(actorInstance: Actor): Unit = {
|
||||
setActorFields(actorInstance, context = null, self = system.deadLetters)
|
||||
|
|
|
|||
|
|
@ -6,17 +6,6 @@ import scala.annotation.tailrec
|
|||
import java.net.MalformedURLException
|
||||
|
||||
object ActorPath {
|
||||
def split(s: String): List[String] = {
|
||||
@tailrec
|
||||
def rec(pos: Int, acc: List[String]): List[String] = {
|
||||
val from = s.lastIndexOf('/', pos - 1)
|
||||
val sub = s.substring(from + 1, pos)
|
||||
val l = sub :: acc
|
||||
if (from == -1) l else rec(from, l)
|
||||
}
|
||||
rec(s.length, Nil)
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse string as actor path; throws java.net.MalformedURLException if unable to do so.
|
||||
*/
|
||||
|
|
@ -25,6 +14,11 @@ object ActorPath {
|
|||
case _ ⇒ throw new MalformedURLException("cannot parse as ActorPath: " + s)
|
||||
}
|
||||
|
||||
/**
|
||||
* This Regular Expression is used to validate a path element (Actor Name).
|
||||
* Since Actors form a tree, it is addressable using an URL, therefor an Actor Name has to conform to:
|
||||
* http://www.ietf.org/rfc/rfc2396.txt
|
||||
*/
|
||||
val ElementRegex = """[-\w:@&=+,.!~*'_;][-\w:@&=+,.!~*'$_;]*""".r
|
||||
}
|
||||
|
||||
|
|
@ -112,21 +106,21 @@ sealed trait ActorPath extends Comparable[ActorPath] with Serializable {
|
|||
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
|
||||
final case class RootActorPath(address: Address, name: String = "/") extends ActorPath {
|
||||
|
||||
def parent: ActorPath = this
|
||||
override def parent: ActorPath = this
|
||||
|
||||
def root: RootActorPath = this
|
||||
override def root: RootActorPath = this
|
||||
|
||||
def /(child: String): ActorPath = new ChildActorPath(this, child)
|
||||
override def /(child: String): ActorPath = new ChildActorPath(this, child)
|
||||
|
||||
val elements: Iterable[String] = List("")
|
||||
override val elements: Iterable[String] = List("")
|
||||
|
||||
override val toString = address + name
|
||||
override val toString: String = address + name
|
||||
|
||||
def toStringWithAddress(addr: Address): String =
|
||||
override def toStringWithAddress(addr: Address): String =
|
||||
if (address.host.isDefined) address + name
|
||||
else addr + name
|
||||
|
||||
def compareTo(other: ActorPath) = other match {
|
||||
override def compareTo(other: ActorPath): Int = other match {
|
||||
case r: RootActorPath ⇒ toString compareTo r.toString
|
||||
case c: ChildActorPath ⇒ 1
|
||||
}
|
||||
|
|
@ -136,11 +130,11 @@ final case class RootActorPath(address: Address, name: String = "/") extends Act
|
|||
final class ChildActorPath(val parent: ActorPath, val name: String) extends ActorPath {
|
||||
if (name.indexOf('/') != -1) throw new IllegalArgumentException("/ is a path separator and is not legal in ActorPath names: [%s]" format name)
|
||||
|
||||
def address: Address = root.address
|
||||
override def address: Address = root.address
|
||||
|
||||
def /(child: String): ActorPath = new ChildActorPath(this, child)
|
||||
override def /(child: String): ActorPath = new ChildActorPath(this, child)
|
||||
|
||||
def elements: Iterable[String] = {
|
||||
override def elements: Iterable[String] = {
|
||||
@tailrec
|
||||
def rec(p: ActorPath, acc: List[String]): Iterable[String] = p match {
|
||||
case r: RootActorPath ⇒ acc
|
||||
|
|
@ -149,7 +143,7 @@ final class ChildActorPath(val parent: ActorPath, val name: String) extends Acto
|
|||
rec(this, Nil)
|
||||
}
|
||||
|
||||
def root = {
|
||||
override def root: RootActorPath = {
|
||||
@tailrec
|
||||
def rec(p: ActorPath): RootActorPath = p match {
|
||||
case r: RootActorPath ⇒ r
|
||||
|
|
@ -209,7 +203,7 @@ final class ChildActorPath(val parent: ActorPath, val name: String) extends Acto
|
|||
finalizeHash(rec(this, startHash(42), startMagicA, startMagicB))
|
||||
}
|
||||
|
||||
def compareTo(other: ActorPath) = {
|
||||
override def compareTo(other: ActorPath): Int = {
|
||||
@tailrec
|
||||
def rec(left: ActorPath, right: ActorPath): Int =
|
||||
if (left eq right) 0
|
||||
|
|
|
|||
|
|
@ -6,7 +6,6 @@ package akka.actor
|
|||
|
||||
import akka.dispatch._
|
||||
import akka.util._
|
||||
import scala.collection.immutable.Stack
|
||||
import java.lang.{ UnsupportedOperationException, IllegalStateException }
|
||||
import akka.serialization.{ Serialization, JavaSerializer }
|
||||
import akka.event.EventStream
|
||||
|
|
@ -160,11 +159,11 @@ trait ScalaActorRef { ref: ActorRef ⇒
|
|||
* often necessary to distinguish between local and non-local references, this
|
||||
* is the only method provided on the scope.
|
||||
*/
|
||||
trait ActorRefScope {
|
||||
private[akka] trait ActorRefScope {
|
||||
def isLocal: Boolean
|
||||
}
|
||||
|
||||
trait LocalRef extends ActorRefScope {
|
||||
private[akka] trait LocalRef extends ActorRefScope {
|
||||
final def isLocal = true
|
||||
}
|
||||
|
||||
|
|
@ -215,18 +214,20 @@ private[akka] abstract class InternalActorRef extends ActorRef with ScalaActorRe
|
|||
* This is an internal look-up failure token, not useful for anything else.
|
||||
*/
|
||||
private[akka] case object Nobody extends MinimalActorRef {
|
||||
val path = new RootActorPath(Address("akka", "all-systems"), "/Nobody")
|
||||
def provider = throw new UnsupportedOperationException("Nobody does not provide")
|
||||
override val path: RootActorPath = new RootActorPath(Address("akka", "all-systems"), "/Nobody")
|
||||
override def provider = throw new UnsupportedOperationException("Nobody does not provide")
|
||||
}
|
||||
|
||||
/**
|
||||
* Local (serializable) ActorRef that is used when referencing the Actor on its "home" node.
|
||||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class LocalActorRef private[akka] (
|
||||
_system: ActorSystemImpl,
|
||||
_props: Props,
|
||||
_supervisor: InternalActorRef,
|
||||
val path: ActorPath,
|
||||
override val path: ActorPath,
|
||||
val systemService: Boolean = false,
|
||||
_receiveTimeout: Option[Duration] = None)
|
||||
extends InternalActorRef with LocalRef {
|
||||
|
|
@ -268,21 +269,21 @@ private[akka] class LocalActorRef private[akka] (
|
|||
* message sends done from the same thread after calling this method will not
|
||||
* be processed until resumed.
|
||||
*/
|
||||
def suspend(): Unit = actorCell.suspend()
|
||||
override def suspend(): Unit = actorCell.suspend()
|
||||
|
||||
/**
|
||||
* Resumes a suspended actor.
|
||||
*/
|
||||
def resume(): Unit = actorCell.resume()
|
||||
override def resume(): Unit = actorCell.resume()
|
||||
|
||||
/**
|
||||
* Shuts down the actor and its message queue
|
||||
*/
|
||||
def stop(): Unit = actorCell.stop()
|
||||
override def stop(): Unit = actorCell.stop()
|
||||
|
||||
def getParent: InternalActorRef = actorCell.parent
|
||||
override def getParent: InternalActorRef = actorCell.parent
|
||||
|
||||
def provider = actorCell.provider
|
||||
override def provider: ActorRefProvider = actorCell.provider
|
||||
|
||||
/**
|
||||
* Method for looking up a single child beneath this actor. Override in order
|
||||
|
|
@ -294,7 +295,7 @@ private[akka] class LocalActorRef private[akka] (
|
|||
case None ⇒ Nobody
|
||||
}
|
||||
|
||||
def getChild(names: Iterator[String]): InternalActorRef = {
|
||||
override def getChild(names: Iterator[String]): InternalActorRef = {
|
||||
/*
|
||||
* The idea is to recursively descend as far as possible with LocalActor
|
||||
* Refs and hand over to that “foreign” child when we encounter it.
|
||||
|
|
@ -303,16 +304,16 @@ private[akka] class LocalActorRef private[akka] (
|
|||
def rec(ref: InternalActorRef, name: Iterator[String]): InternalActorRef =
|
||||
ref match {
|
||||
case l: LocalActorRef ⇒
|
||||
val n = name.next()
|
||||
val next = n match {
|
||||
val next = name.next() match {
|
||||
case ".." ⇒ l.getParent
|
||||
case "" ⇒ l
|
||||
case _ ⇒ l.getSingleChild(n)
|
||||
case any ⇒ l.getSingleChild(any)
|
||||
}
|
||||
if (next == Nobody || name.isEmpty) next else rec(next, name)
|
||||
case _ ⇒
|
||||
ref.getChild(name)
|
||||
}
|
||||
|
||||
if (names.isEmpty) this
|
||||
else rec(this, names)
|
||||
}
|
||||
|
|
@ -321,11 +322,11 @@ private[akka] class LocalActorRef private[akka] (
|
|||
|
||||
protected[akka] def underlying: ActorCell = actorCell
|
||||
|
||||
def sendSystemMessage(message: SystemMessage) { underlying.dispatcher.systemDispatch(underlying, message) }
|
||||
override def sendSystemMessage(message: SystemMessage): Unit = underlying.dispatcher.systemDispatch(underlying, message)
|
||||
|
||||
def !(message: Any)(implicit sender: ActorRef = null): Unit = actorCell.tell(message, sender)
|
||||
override def !(message: Any)(implicit sender: ActorRef = null): Unit = actorCell.tell(message, sender)
|
||||
|
||||
def restart(cause: Throwable): Unit = actorCell.restart(cause)
|
||||
override def restart(cause: Throwable): Unit = actorCell.restart(cause)
|
||||
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
protected def writeReplace(): AnyRef = SerializedActorRef(path)
|
||||
|
|
@ -333,9 +334,10 @@ private[akka] class LocalActorRef private[akka] (
|
|||
|
||||
/**
|
||||
* Memento pattern for serializing ActorRefs transparently
|
||||
* INTERNAL API
|
||||
*/
|
||||
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
|
||||
case class SerializedActorRef private (path: String) {
|
||||
private[akka] case class SerializedActorRef private (path: String) {
|
||||
import akka.serialization.JavaSerializer.currentSystem
|
||||
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
|
|
@ -349,7 +351,10 @@ case class SerializedActorRef private (path: String) {
|
|||
}
|
||||
}
|
||||
|
||||
object SerializedActorRef {
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object SerializedActorRef {
|
||||
def apply(path: ActorPath): SerializedActorRef = {
|
||||
Serialization.currentTransportAddress.value match {
|
||||
case null ⇒ new SerializedActorRef(path.toString)
|
||||
|
|
@ -360,33 +365,32 @@ object SerializedActorRef {
|
|||
|
||||
/**
|
||||
* Trait for ActorRef implementations where all methods contain default stubs.
|
||||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] trait MinimalActorRef extends InternalActorRef with LocalRef {
|
||||
|
||||
def getParent: InternalActorRef = Nobody
|
||||
override def getParent: InternalActorRef = Nobody
|
||||
override def getChild(names: Iterator[String]): InternalActorRef = if (names.forall(_.isEmpty)) this else Nobody
|
||||
|
||||
def getChild(names: Iterator[String]): InternalActorRef = {
|
||||
val dropped = names.dropWhile(_.isEmpty)
|
||||
if (dropped.isEmpty) this
|
||||
else Nobody
|
||||
}
|
||||
override def suspend(): Unit = ()
|
||||
override def resume(): Unit = ()
|
||||
override def stop(): Unit = ()
|
||||
override def isTerminated = false
|
||||
|
||||
def suspend(): Unit = ()
|
||||
def resume(): Unit = ()
|
||||
override def !(message: Any)(implicit sender: ActorRef = null): Unit = ()
|
||||
|
||||
def stop(): Unit = ()
|
||||
|
||||
def isTerminated = false
|
||||
|
||||
def !(message: Any)(implicit sender: ActorRef = null): Unit = ()
|
||||
|
||||
def sendSystemMessage(message: SystemMessage): Unit = ()
|
||||
def restart(cause: Throwable): Unit = ()
|
||||
override def sendSystemMessage(message: SystemMessage): Unit = ()
|
||||
override def restart(cause: Throwable): Unit = ()
|
||||
|
||||
@throws(classOf[java.io.ObjectStreamException])
|
||||
protected def writeReplace(): AnyRef = SerializedActorRef(path)
|
||||
}
|
||||
|
||||
/**
|
||||
* When a message is sent to an Actor that is terminated before receiving the message, it will be sent as a DeadLetter
|
||||
* to the ActorSystem's EventStream
|
||||
*/
|
||||
case class DeadLetter(message: Any, sender: ActorRef, recipient: ActorRef)
|
||||
|
||||
private[akka] object DeadLetterActorRef {
|
||||
|
|
@ -402,10 +406,12 @@ private[akka] object DeadLetterActorRef {
|
|||
/**
|
||||
* This special dead letter reference has a name: it is that which is returned
|
||||
* by a local look-up which is unsuccessful.
|
||||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class EmptyLocalActorRef(
|
||||
val provider: ActorRefProvider,
|
||||
val path: ActorPath,
|
||||
override val provider: ActorRefProvider,
|
||||
override val path: ActorPath,
|
||||
val eventStream: EventStream) extends MinimalActorRef {
|
||||
|
||||
override def isTerminated(): Boolean = true
|
||||
|
|
@ -419,6 +425,8 @@ private[akka] class EmptyLocalActorRef(
|
|||
/**
|
||||
* Internal implementation of the dead letter destination: will publish any
|
||||
* received message to the eventStream, wrapped as [[akka.actor.DeadLetter]].
|
||||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class DeadLetterActorRef(_provider: ActorRefProvider, _path: ActorPath, _eventStream: EventStream)
|
||||
extends EmptyLocalActorRef(_provider, _path, _eventStream) {
|
||||
|
|
@ -434,10 +442,12 @@ private[akka] class DeadLetterActorRef(_provider: ActorRefProvider, _path: Actor
|
|||
|
||||
/**
|
||||
* Internal implementation detail used for paths like “/temp”
|
||||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class VirtualPathContainer(
|
||||
val provider: ActorRefProvider,
|
||||
val path: ActorPath,
|
||||
override val provider: ActorRefProvider,
|
||||
override val path: ActorPath,
|
||||
override val getParent: InternalActorRef,
|
||||
val log: LoggingAdapter) extends MinimalActorRef {
|
||||
|
||||
|
|
@ -450,12 +460,8 @@ private[akka] class VirtualPathContainer(
|
|||
}
|
||||
}
|
||||
|
||||
def removeChild(name: String): Unit = {
|
||||
children.remove(name) match {
|
||||
case null ⇒ log.warning("{} trying to remove non-child {}", path, name)
|
||||
case _ ⇒ //okay
|
||||
}
|
||||
}
|
||||
def removeChild(name: String): Unit =
|
||||
if (children.remove(name) eq null) log.warning("{} trying to remove non-child {}", path, name)
|
||||
|
||||
def getChild(name: String): InternalActorRef = children.get(name)
|
||||
|
||||
|
|
|
|||
|
|
@ -49,8 +49,12 @@ trait ActorRefProvider {
|
|||
*/
|
||||
def rootPath: ActorPath
|
||||
|
||||
/**
|
||||
* The Settings associated with this ActorRefProvider
|
||||
*/
|
||||
def settings: ActorSystem.Settings
|
||||
|
||||
//FIXME WHY IS THIS HERE?
|
||||
def dispatcher: MessageDispatcher
|
||||
|
||||
/**
|
||||
|
|
@ -61,8 +65,12 @@ trait ActorRefProvider {
|
|||
*/
|
||||
def init(system: ActorSystemImpl): Unit
|
||||
|
||||
/**
|
||||
* The Deployer associated with this ActorRefProvider
|
||||
*/
|
||||
def deployer: Deployer
|
||||
|
||||
//FIXME WHY IS THIS HERE?
|
||||
def scheduler: Scheduler
|
||||
|
||||
/**
|
||||
|
|
@ -131,6 +139,7 @@ trait ActorRefProvider {
|
|||
*/
|
||||
def terminationFuture: Future[Unit]
|
||||
|
||||
//FIXME I PROPOSE TO REMOVE THIS IN 2.1 - √
|
||||
/**
|
||||
* Obtain the address which is to be used within sender references when
|
||||
* sending to the given other address or none if the other address cannot be
|
||||
|
|
@ -141,22 +150,33 @@ trait ActorRefProvider {
|
|||
}
|
||||
|
||||
/**
|
||||
* Interface implemented by ActorSystem and AkkaContext, the only two places
|
||||
* Interface implemented by ActorSystem and ActorContext, the only two places
|
||||
* from which you can get fresh actors.
|
||||
*/
|
||||
trait ActorRefFactory {
|
||||
|
||||
/**
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
protected def systemImpl: ActorSystemImpl
|
||||
|
||||
/**
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
protected def provider: ActorRefProvider
|
||||
|
||||
/**
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
protected def dispatcher: MessageDispatcher
|
||||
|
||||
/**
|
||||
* Father of all children created by this interface.
|
||||
*
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
protected def guardian: InternalActorRef
|
||||
|
||||
/**
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
protected def lookupRoot: InternalActorRef
|
||||
|
||||
/**
|
||||
|
|
@ -276,8 +296,6 @@ trait ActorRefFactory {
|
|||
def stop(actor: ActorRef): Unit
|
||||
}
|
||||
|
||||
class ActorRefProviderException(message: String) extends AkkaException(message)
|
||||
|
||||
/**
|
||||
* Internal Akka use only, used in implementation of system.actorOf.
|
||||
*/
|
||||
|
|
@ -298,10 +316,10 @@ private[akka] case class StopChild(child: ActorRef)
|
|||
*/
|
||||
class LocalActorRefProvider(
|
||||
_systemName: String,
|
||||
val settings: ActorSystem.Settings,
|
||||
override val settings: ActorSystem.Settings,
|
||||
val eventStream: EventStream,
|
||||
val scheduler: Scheduler,
|
||||
val deployer: Deployer) extends ActorRefProvider {
|
||||
override val scheduler: Scheduler,
|
||||
override val deployer: Deployer) extends ActorRefProvider {
|
||||
|
||||
// this is the constructor needed for reflectively instantiating the provider
|
||||
def this(_systemName: String,
|
||||
|
|
@ -315,13 +333,13 @@ class LocalActorRefProvider(
|
|||
scheduler,
|
||||
new Deployer(settings, dynamicAccess))
|
||||
|
||||
val rootPath: ActorPath = RootActorPath(Address("akka", _systemName))
|
||||
override val rootPath: ActorPath = RootActorPath(Address("akka", _systemName))
|
||||
|
||||
val log = Logging(eventStream, "LocalActorRefProvider(" + rootPath.address + ")")
|
||||
private[akka] val log: LoggingAdapter = Logging(eventStream, "LocalActorRefProvider(" + rootPath.address + ")")
|
||||
|
||||
val deadLetters = new DeadLetterActorRef(this, rootPath / "deadLetters", eventStream)
|
||||
override val deadLetters: InternalActorRef = new DeadLetterActorRef(this, rootPath / "deadLetters", eventStream)
|
||||
|
||||
val deathWatch = new LocalDeathWatch(1024) //TODO make configrable
|
||||
override val deathWatch: DeathWatch = new LocalDeathWatch(1024) //TODO make configrable
|
||||
|
||||
/*
|
||||
* generate name for temporary actor refs
|
||||
|
|
@ -332,7 +350,7 @@ class LocalActorRefProvider(
|
|||
|
||||
private val tempNode = rootPath / "temp"
|
||||
|
||||
def tempPath() = tempNode / tempName()
|
||||
override def tempPath(): ActorPath = tempNode / tempName()
|
||||
|
||||
/**
|
||||
* Top-level anchor for the supervision hierarchy of this actor system. Will
|
||||
|
|
@ -348,11 +366,11 @@ class LocalActorRefProvider(
|
|||
|
||||
def provider: ActorRefProvider = LocalActorRefProvider.this
|
||||
|
||||
override def stop() = stopped switchOn {
|
||||
override def stop(): Unit = stopped switchOn {
|
||||
terminationFuture.complete(causeOfTermination.toLeft(()))
|
||||
}
|
||||
|
||||
override def isTerminated = stopped.isOn
|
||||
override def isTerminated: Boolean = stopped.isOn
|
||||
|
||||
override def !(message: Any)(implicit sender: ActorRef = null): Unit = stopped.ifOff(message match {
|
||||
case Failed(ex) if sender ne null ⇒ causeOfTermination = Some(ex); sender.asInstanceOf[InternalActorRef].stop()
|
||||
|
|
@ -371,7 +389,7 @@ class LocalActorRefProvider(
|
|||
/**
|
||||
* Overridable supervision strategy to be used by the “/user” guardian.
|
||||
*/
|
||||
protected def guardianSupervisionStrategy = {
|
||||
protected def guardianSupervisionStrategy: SupervisorStrategy = {
|
||||
import akka.actor.SupervisorStrategy._
|
||||
OneForOneStrategy() {
|
||||
case _: ActorKilledException ⇒ Stop
|
||||
|
|
@ -387,12 +405,12 @@ class LocalActorRefProvider(
|
|||
*/
|
||||
private class Guardian extends Actor {
|
||||
|
||||
override val supervisorStrategy = guardianSupervisionStrategy
|
||||
override val supervisorStrategy: SupervisorStrategy = guardianSupervisionStrategy
|
||||
|
||||
def receive = {
|
||||
case Terminated(_) ⇒ context.stop(self)
|
||||
case CreateChild(child, name) ⇒ sender ! (try context.actorOf(child, name) catch { case e: Exception ⇒ e })
|
||||
case CreateRandomNameChild(child) ⇒ sender ! (try context.actorOf(child) catch { case e: Exception ⇒ e })
|
||||
case CreateChild(child, name) ⇒ sender ! (try context.actorOf(child, name) catch { case e: Exception ⇒ e }) // FIXME shouldn't this use NonFatal & Status.Failure?
|
||||
case CreateRandomNameChild(child) ⇒ sender ! (try context.actorOf(child) catch { case e: Exception ⇒ e }) // FIXME shouldn't this use NonFatal & Status.Failure?
|
||||
case StopChild(child) ⇒ context.stop(child); sender ! "ok"
|
||||
case m ⇒ deadLetters ! DeadLetter(m, sender, self)
|
||||
}
|
||||
|
|
@ -404,12 +422,11 @@ class LocalActorRefProvider(
|
|||
/**
|
||||
* Overridable supervision strategy to be used by the “/system” guardian.
|
||||
*/
|
||||
protected def systemGuardianSupervisionStrategy = {
|
||||
protected def systemGuardianSupervisionStrategy: SupervisorStrategy = {
|
||||
import akka.actor.SupervisorStrategy._
|
||||
OneForOneStrategy() {
|
||||
case _: ActorKilledException ⇒ Stop
|
||||
case _: ActorInitializationException ⇒ Stop
|
||||
case _: Exception ⇒ Restart
|
||||
case _: ActorKilledException | _: ActorInitializationException ⇒ Stop
|
||||
case _: Exception ⇒ Restart
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -420,14 +437,12 @@ class LocalActorRefProvider(
|
|||
*/
|
||||
private class SystemGuardian extends Actor {
|
||||
|
||||
override val supervisorStrategy = systemGuardianSupervisionStrategy
|
||||
override val supervisorStrategy: SupervisorStrategy = systemGuardianSupervisionStrategy
|
||||
|
||||
def receive = {
|
||||
case Terminated(_) ⇒
|
||||
eventStream.stopDefaultLoggers()
|
||||
context.stop(self)
|
||||
case CreateChild(child, name) ⇒ sender ! (try context.actorOf(child, name) catch { case e: Exception ⇒ e })
|
||||
case CreateRandomNameChild(child) ⇒ sender ! (try context.actorOf(child) catch { case e: Exception ⇒ e })
|
||||
case Terminated(_) ⇒ eventStream.stopDefaultLoggers(); context.stop(self)
|
||||
case CreateChild(child, name) ⇒ sender ! (try context.actorOf(child, name) catch { case e: Exception ⇒ e }) // FIXME shouldn't this use NonFatal & Status.Failure?
|
||||
case CreateRandomNameChild(child) ⇒ sender ! (try context.actorOf(child) catch { case e: Exception ⇒ e }) // FIXME shouldn't this use NonFatal & Status.Failure?
|
||||
case StopChild(child) ⇒ context.stop(child); sender ! "ok"
|
||||
case m ⇒ deadLetters ! DeadLetter(m, sender, self)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -5,6 +5,10 @@ package akka.actor
|
|||
import java.util.regex.Pattern
|
||||
import akka.util.Helpers
|
||||
|
||||
/**
|
||||
* An ActorSelection is a logical view of a section of an ActorSystem's tree of Actors,
|
||||
* allowing for broadcasting of messages to that section.
|
||||
*/
|
||||
abstract class ActorSelection {
|
||||
this: ScalaActorSelection ⇒
|
||||
|
||||
|
|
@ -12,11 +16,11 @@ abstract class ActorSelection {
|
|||
|
||||
protected def path: Array[AnyRef]
|
||||
|
||||
def tell(msg: Any) { target ! toMessage(msg, path) }
|
||||
def tell(msg: Any): Unit = target ! toMessage(msg, path)
|
||||
|
||||
def tell(msg: Any, sender: ActorRef) { target.tell(toMessage(msg, path), sender) }
|
||||
def tell(msg: Any, sender: ActorRef): Unit = target.tell(toMessage(msg, path), sender)
|
||||
|
||||
// this may want to be fast ...
|
||||
// FIXME make this so that "next" instead is the remaining path
|
||||
private def toMessage(msg: Any, path: Array[AnyRef]): Any = {
|
||||
var acc = msg
|
||||
var index = path.length - 1
|
||||
|
|
@ -32,7 +36,12 @@ abstract class ActorSelection {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* An ActorSelection is a logical view of a section of an ActorSystem's tree of Actors,
|
||||
* allowing for broadcasting of messages to that section.
|
||||
*/
|
||||
object ActorSelection {
|
||||
//This cast is safe because the self-type of ActorSelection requires that it mixes in ScalaActorSelection
|
||||
implicit def toScala(sel: ActorSelection): ScalaActorSelection = sel.asInstanceOf[ScalaActorSelection]
|
||||
|
||||
/**
|
||||
|
|
@ -43,7 +52,7 @@ object ActorSelection {
|
|||
*/
|
||||
def apply(anchor: ActorRef, path: String): ActorSelection = {
|
||||
val elems = path.split("/+").dropWhile(_.isEmpty)
|
||||
val compiled: Array[AnyRef] = elems map (x ⇒ if (x.contains("?") || x.contains("*")) Helpers.makePattern(x) else x)
|
||||
val compiled: Array[AnyRef] = elems map (x ⇒ if ((x.indexOf('?') != -1) || (x.indexOf('*') != -1)) Helpers.makePattern(x) else x)
|
||||
new ActorSelection with ScalaActorSelection {
|
||||
def target = anchor
|
||||
def path = compiled
|
||||
|
|
@ -51,6 +60,10 @@ object ActorSelection {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Contains the Scala API (!-method) for ActorSelections) which provides automatic tracking of the sender,
|
||||
* as per the usual implicit ActorRef pattern.
|
||||
*/
|
||||
trait ScalaActorSelection {
|
||||
this: ActorSelection ⇒
|
||||
|
||||
|
|
|
|||
|
|
@ -4,38 +4,34 @@
|
|||
|
||||
package akka.actor
|
||||
|
||||
import akka.config.ConfigurationException
|
||||
import akka.event._
|
||||
import akka.dispatch._
|
||||
import akka.pattern.ask
|
||||
import org.jboss.netty.akka.util.HashedWheelTimer
|
||||
import java.util.concurrent.TimeUnit.MILLISECONDS
|
||||
import com.typesafe.config.Config
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import com.typesafe.config.{ Config, ConfigFactory }
|
||||
import scala.annotation.tailrec
|
||||
import org.jboss.netty.akka.util.internal.ConcurrentIdentityHashMap
|
||||
import java.io.Closeable
|
||||
import akka.dispatch.Await.Awaitable
|
||||
import akka.dispatch.Await.CanAwait
|
||||
import akka.dispatch.Await.{ Awaitable, CanAwait }
|
||||
import akka.util._
|
||||
import akka.util.internal.{ HashedWheelTimer, ConcurrentIdentityHashMap }
|
||||
import collection.immutable.Stack
|
||||
import java.util.concurrent.{ ThreadFactory, CountDownLatch, TimeoutException, RejectedExecutionException }
|
||||
import java.util.concurrent.TimeUnit.MILLISECONDS
|
||||
|
||||
object ActorSystem {
|
||||
|
||||
val Version = "2.1-SNAPSHOT"
|
||||
val Version: String = "2.1-SNAPSHOT"
|
||||
|
||||
val EnvHome = System.getenv("AKKA_HOME") match {
|
||||
val EnvHome: Option[String] = System.getenv("AKKA_HOME") match {
|
||||
case null | "" | "." ⇒ None
|
||||
case value ⇒ Some(value)
|
||||
}
|
||||
|
||||
val SystemHome = System.getProperty("akka.home") match {
|
||||
val SystemHome: Option[String] = System.getProperty("akka.home") match {
|
||||
case null | "" ⇒ None
|
||||
case value ⇒ Some(value)
|
||||
}
|
||||
|
||||
val GlobalHome = SystemHome orElse EnvHome
|
||||
val GlobalHome: Option[String] = SystemHome orElse EnvHome
|
||||
|
||||
/**
|
||||
* Creates a new ActorSystem with the name "default",
|
||||
|
|
@ -102,8 +98,16 @@ object ActorSystem {
|
|||
*/
|
||||
def apply(name: String, config: Config, classLoader: ClassLoader): ActorSystem = new ActorSystemImpl(name, config, classLoader).start()
|
||||
|
||||
/**
|
||||
* Settings are the overall ActorSystem Settings which also provides a convenient access to the Config object.
|
||||
*
|
||||
* For more detailed information about the different possible configuration options, look in the Akka Documentation under "Configuration"
|
||||
*/
|
||||
class Settings(classLoader: ClassLoader, cfg: Config, final val name: String) {
|
||||
|
||||
/**
|
||||
* The backing Config of this ActorSystem's Settings
|
||||
*/
|
||||
final val config: Config = {
|
||||
val config = cfg.withFallback(ConfigFactory.defaultReference(classLoader))
|
||||
config.checkValid(ConfigFactory.defaultReference(classLoader), "akka")
|
||||
|
|
@ -114,11 +118,9 @@ object ActorSystem {
|
|||
import config._
|
||||
|
||||
final val ConfigVersion = getString("akka.version")
|
||||
|
||||
final val ProviderClass = getString("akka.actor.provider")
|
||||
|
||||
final val CreationTimeout = Timeout(Duration(getMilliseconds("akka.actor.creation-timeout"), MILLISECONDS))
|
||||
final val ReaperInterval = Duration(getMilliseconds("akka.actor.reaper-interval"), MILLISECONDS)
|
||||
|
||||
final val SerializeAllMessages = getBoolean("akka.actor.serialize-messages")
|
||||
final val SerializeAllCreators = getBoolean("akka.actor.serialize-creators")
|
||||
|
||||
|
|
@ -146,13 +148,16 @@ object ActorSystem {
|
|||
final val JvmExitOnFatalError = getBoolean("akka.jvm-exit-on-fatal-error")
|
||||
|
||||
if (ConfigVersion != Version)
|
||||
throw new ConfigurationException("Akka JAR version [" + Version + "] does not match the provided config version [" + ConfigVersion + "]")
|
||||
throw new akka.ConfigurationException("Akka JAR version [" + Version + "] does not match the provided config version [" + ConfigVersion + "]")
|
||||
|
||||
/**
|
||||
* Returns the String representation of the Config that this Settings is backed by
|
||||
*/
|
||||
override def toString: String = config.root.render
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
private[akka] def findClassLoader(): ClassLoader = {
|
||||
def findCaller(get: Int ⇒ Class[_]): ClassLoader =
|
||||
|
|
@ -423,7 +428,7 @@ abstract class ExtendedActorSystem extends ActorSystem {
|
|||
def dynamicAccess: DynamicAccess
|
||||
}
|
||||
|
||||
class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Config, classLoader: ClassLoader) extends ExtendedActorSystem {
|
||||
private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config, classLoader: ClassLoader) extends ExtendedActorSystem {
|
||||
|
||||
if (!name.matches("""^[a-zA-Z0-9][a-zA-Z0-9-]*$"""))
|
||||
throw new IllegalArgumentException(
|
||||
|
|
@ -476,7 +481,7 @@ class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Conf
|
|||
|
||||
def logConfiguration(): Unit = log.info(settings.toString)
|
||||
|
||||
protected def systemImpl = this
|
||||
protected def systemImpl: ActorSystemImpl = this
|
||||
|
||||
private[akka] def systemActorOf(props: Props, name: String): ActorRef = {
|
||||
implicit val timeout = settings.CreationTimeout
|
||||
|
|
@ -540,6 +545,7 @@ class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Conf
|
|||
|
||||
def deadLetters: ActorRef = provider.deadLetters
|
||||
|
||||
//FIXME Why do we need this at all?
|
||||
val deadLetterQueue: MessageQueue = new MessageQueue {
|
||||
def enqueue(receiver: ActorRef, envelope: Envelope) { deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver) }
|
||||
def dequeue() = null
|
||||
|
|
@ -547,7 +553,7 @@ class ActorSystemImpl protected[akka] (val name: String, applicationConfig: Conf
|
|||
def numberOfMessages = 0
|
||||
def cleanUp(owner: ActorContext, deadLetters: MessageQueue): Unit = ()
|
||||
}
|
||||
|
||||
//FIXME Why do we need this at all?
|
||||
val deadLetterMailbox: Mailbox = new Mailbox(null, deadLetterQueue) {
|
||||
becomeClosed()
|
||||
def systemEnqueue(receiver: ActorRef, handle: SystemMessage): Unit = deadLetters ! DeadLetter(handle, receiver, receiver)
|
||||
|
|
|
|||
|
|
@ -5,6 +5,7 @@ package akka.actor
|
|||
import java.net.URI
|
||||
import java.net.URISyntaxException
|
||||
import java.net.MalformedURLException
|
||||
import annotation.tailrec
|
||||
|
||||
/**
|
||||
* The address specifies the physical location under which an Actor can be
|
||||
|
|
@ -20,36 +21,62 @@ final case class Address private (protocol: String, system: String, host: Option
|
|||
def this(protocol: String, system: String) = this(protocol, system, None, None)
|
||||
def this(protocol: String, system: String, host: String, port: Int) = this(protocol, system, Option(host), Some(port))
|
||||
|
||||
/**
|
||||
* Returns the canonical String representation of this Address formatted as:
|
||||
*
|
||||
* <protocol>://<system>@<host>:<port>
|
||||
*/
|
||||
@transient
|
||||
override lazy val toString: String = {
|
||||
val sb = new StringBuilder(protocol)
|
||||
sb.append("://")
|
||||
sb.append(system)
|
||||
if (host.isDefined) {
|
||||
sb.append('@')
|
||||
sb.append(host.get)
|
||||
}
|
||||
if (port.isDefined) {
|
||||
sb.append(':')
|
||||
sb.append(port.get)
|
||||
}
|
||||
val sb = (new StringBuilder(protocol)).append("://").append(system)
|
||||
|
||||
if (host.isDefined) sb.append('@').append(host.get)
|
||||
if (port.isDefined) sb.append(':').append(port.get)
|
||||
|
||||
sb.toString
|
||||
}
|
||||
|
||||
def hostPort: String = toString.substring(protocol.length() + 3)
|
||||
/**
|
||||
* Returns a String representation formatted as:
|
||||
*
|
||||
* <system>@<host>:<port>
|
||||
*/
|
||||
def hostPort: String = toString.substring(protocol.length + 3)
|
||||
}
|
||||
|
||||
object Address {
|
||||
/**
|
||||
* Constructs a new Address with the specified protocol and system name
|
||||
*/
|
||||
def apply(protocol: String, system: String) = new Address(protocol, system)
|
||||
|
||||
/**
|
||||
* Constructs a new Address with the specified protocol, system name, host and port
|
||||
*/
|
||||
def apply(protocol: String, system: String, host: String, port: Int) = new Address(protocol, system, Some(host), Some(port))
|
||||
}
|
||||
|
||||
object RelativeActorPath {
|
||||
private[akka] trait PathUtils {
|
||||
protected def split(s: String): List[String] = {
|
||||
@tailrec
|
||||
def rec(pos: Int, acc: List[String]): List[String] = {
|
||||
val from = s.lastIndexOf('/', pos - 1)
|
||||
val sub = s.substring(from + 1, pos)
|
||||
val l = sub :: acc
|
||||
if (from == -1) l else rec(from, l)
|
||||
}
|
||||
rec(s.length, Nil)
|
||||
}
|
||||
}
|
||||
|
||||
object RelativeActorPath extends PathUtils {
|
||||
def unapply(addr: String): Option[Iterable[String]] = {
|
||||
try {
|
||||
val uri = new URI(addr)
|
||||
if (uri.isAbsolute) None
|
||||
else Some(ActorPath.split(uri.getPath))
|
||||
else Some(split(uri.getPath))
|
||||
} catch {
|
||||
case _: URISyntaxException ⇒ None
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -58,13 +85,7 @@ object RelativeActorPath {
|
|||
* This object serves as extractor for Scala and as address parser for Java.
|
||||
*/
|
||||
object AddressFromURIString {
|
||||
def unapply(addr: String): Option[Address] =
|
||||
try {
|
||||
val uri = new URI(addr)
|
||||
unapply(uri)
|
||||
} catch {
|
||||
case _: URISyntaxException ⇒ None
|
||||
}
|
||||
def unapply(addr: String): Option[Address] = try unapply(new URI(addr)) catch { case _: URISyntaxException ⇒ None }
|
||||
|
||||
def unapply(uri: URI): Option[Address] =
|
||||
if (uri eq null) None
|
||||
|
|
@ -84,7 +105,7 @@ object AddressFromURIString {
|
|||
*/
|
||||
def apply(addr: String): Address = addr match {
|
||||
case AddressFromURIString(address) ⇒ address
|
||||
case _ ⇒ throw new MalformedURLException
|
||||
case _ ⇒ throw new MalformedURLException(addr)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -93,14 +114,17 @@ object AddressFromURIString {
|
|||
def parse(addr: String): Address = apply(addr)
|
||||
}
|
||||
|
||||
object ActorPathExtractor {
|
||||
/**
|
||||
* Given an ActorPath it returns the Address and the path elements if the path is well-formed
|
||||
*/
|
||||
object ActorPathExtractor extends PathUtils {
|
||||
def unapply(addr: String): Option[(Address, Iterable[String])] =
|
||||
try {
|
||||
val uri = new URI(addr)
|
||||
if (uri.getPath == null) None
|
||||
else AddressFromURIString.unapply(uri) match {
|
||||
case None ⇒ None
|
||||
case Some(addr) ⇒ Some((addr, ActorPath.split(uri.getPath).drop(1)))
|
||||
case Some(addr) ⇒ Some((addr, split(uri.getPath).drop(1)))
|
||||
}
|
||||
} catch {
|
||||
case _: URISyntaxException ⇒ None
|
||||
|
|
|
|||
|
|
@ -34,8 +34,19 @@ final case class Deploy(
|
|||
routerConfig: RouterConfig = NoRouter,
|
||||
scope: Scope = NoScopeGiven) {
|
||||
|
||||
/**
|
||||
* Java API to create a Deploy with the given RouterConfig
|
||||
*/
|
||||
def this(routing: RouterConfig) = this("", ConfigFactory.empty, routing)
|
||||
|
||||
/**
|
||||
* Java API to create a Deploy with the given RouterConfig with Scope
|
||||
*/
|
||||
def this(routing: RouterConfig, scope: Scope) = this("", ConfigFactory.empty, routing, scope)
|
||||
|
||||
/**
|
||||
* Java API to create a Deploy with the given Scope
|
||||
*/
|
||||
def this(scope: Scope) = this("", ConfigFactory.empty, NoRouter, scope)
|
||||
|
||||
/**
|
||||
|
|
@ -67,13 +78,9 @@ trait Scope {
|
|||
|
||||
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
|
||||
abstract class LocalScope extends Scope
|
||||
case object LocalScope extends LocalScope {
|
||||
/**
|
||||
* Java API
|
||||
*/
|
||||
@deprecated("use instance() method instead", "2.0.1")
|
||||
def scope: Scope = this
|
||||
|
||||
//FIXME docs
|
||||
case object LocalScope extends LocalScope {
|
||||
/**
|
||||
* Java API: get the singleton instance
|
||||
*/
|
||||
|
|
@ -128,7 +135,7 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce
|
|||
add(d.path.split("/").drop(1), d)
|
||||
}
|
||||
|
||||
protected def parseConfig(key: String, config: Config): Option[Deploy] = {
|
||||
def parseConfig(key: String, config: Config): Option[Deploy] = {
|
||||
|
||||
val deployment = config.withFallback(default)
|
||||
|
||||
|
|
@ -162,5 +169,4 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce
|
|||
|
||||
Some(Deploy(key, deployment, router, NoScopeGiven))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -14,7 +14,7 @@ import java.lang.reflect.InvocationTargetException
|
|||
* This is an internal facility and users are not expected to encounter it
|
||||
* unless they are extending Akka in ways which go beyond simple Extensions.
|
||||
*/
|
||||
trait DynamicAccess {
|
||||
abstract class DynamicAccess {
|
||||
|
||||
/**
|
||||
* Convenience method which given a `Class[_]` object and a constructor description
|
||||
|
|
@ -88,7 +88,7 @@ trait DynamicAccess {
|
|||
* by default.
|
||||
*/
|
||||
class ReflectiveDynamicAccess(val classLoader: ClassLoader) extends DynamicAccess {
|
||||
|
||||
//FIXME switch to Scala Reflection for 2.10
|
||||
override def getClassFor[T: ClassManifest](fqcn: String): Either[Throwable, Class[_ <: T]] =
|
||||
try {
|
||||
val c = classLoader.loadClass(fqcn).asInstanceOf[Class[_ <: T]]
|
||||
|
|
|
|||
|
|
@ -6,30 +6,86 @@ package akka.actor
|
|||
import akka.util._
|
||||
|
||||
import scala.collection.mutable
|
||||
import akka.event.Logging
|
||||
import akka.routing.{ Deafen, Listen, Listeners }
|
||||
|
||||
object FSM {
|
||||
|
||||
/**
|
||||
* A partial function value which does not match anything and can be used to
|
||||
* “reset” `whenUnhandled` and `onTermination` handlers.
|
||||
*
|
||||
* {{{
|
||||
* onTermination(FSM.NullFunction)
|
||||
* }}}
|
||||
*/
|
||||
object NullFunction extends PartialFunction[Any, Nothing] {
|
||||
def isDefinedAt(o: Any) = false
|
||||
def apply(o: Any) = sys.error("undefined")
|
||||
}
|
||||
|
||||
/**
|
||||
* Message type which is sent directly to the subscribed actor in
|
||||
* [[akka.actor.FSM.SubscribeTransitionCallback]] before sending any
|
||||
* [[akka.actor.FSM.Transition]] messages.
|
||||
*/
|
||||
case class CurrentState[S](fsmRef: ActorRef, state: S)
|
||||
|
||||
/**
|
||||
* Message type which is used to communicate transitions between states to
|
||||
* all subscribed listeners (use [[akka.actor.FSM.SubscribeTransitionCallback]]).
|
||||
*/
|
||||
case class Transition[S](fsmRef: ActorRef, from: S, to: S)
|
||||
|
||||
/**
|
||||
* Send this to an [[akka.actor.FSM]] to request first the [[akka.actor.CurrentState]]
|
||||
* and then a series of [[akka.actor.Transition]] updates. Cancel the subscription
|
||||
* using [[akka.actor.FSM.UnsubscribeTransitionCallback]].
|
||||
*/
|
||||
case class SubscribeTransitionCallBack(actorRef: ActorRef)
|
||||
|
||||
/**
|
||||
* Unsubscribe from [[akka.actor.FSM.Transition]] notifications which was
|
||||
* effected by sending the corresponding [[akka.actor.FSM.SubscribeTransitionCallback]].
|
||||
*/
|
||||
case class UnsubscribeTransitionCallBack(actorRef: ActorRef)
|
||||
|
||||
/**
|
||||
* Reason why this [[akka.actor.FSM]] is shutting down.
|
||||
*/
|
||||
sealed trait Reason
|
||||
|
||||
/**
|
||||
* Default reason if calling `stop()`.
|
||||
*/
|
||||
case object Normal extends Reason
|
||||
|
||||
/**
|
||||
* Reason given when someone was calling `system.stop(fsm)` from outside;
|
||||
* also applies to `Stop` supervision directive.
|
||||
*/
|
||||
case object Shutdown extends Reason
|
||||
|
||||
/**
|
||||
* Signifies that the [[akka.actor.FSM]] is shutting itself down because of
|
||||
* an error, e.g. if the state to transition into does not exist. You can use
|
||||
* this to communicate a more precise cause to the [[akka.actor.FSM$onTermination]] block.
|
||||
*/
|
||||
case class Failure(cause: Any) extends Reason
|
||||
|
||||
/**
|
||||
* This case object is received in case of a state timeout.
|
||||
*/
|
||||
case object StateTimeout
|
||||
case class TimeoutMarker(generation: Long)
|
||||
|
||||
case class Timer(name: String, msg: Any, repeat: Boolean, generation: Int)(implicit system: ActorSystem) {
|
||||
/**
|
||||
* Internal API
|
||||
*/
|
||||
private case class TimeoutMarker(generation: Long)
|
||||
|
||||
/**
|
||||
* Internal API
|
||||
*/
|
||||
private[akka] case class Timer(name: String, msg: Any, repeat: Boolean, generation: Int)(implicit system: ActorSystem) {
|
||||
private var ref: Option[Cancellable] = _
|
||||
|
||||
def schedule(actor: ActorRef, timeout: Duration) {
|
||||
|
|
@ -56,8 +112,16 @@ object FSM {
|
|||
def unapply[S](in: (S, S)) = Some(in)
|
||||
}
|
||||
|
||||
/**
|
||||
* Log Entry of the [[akka.actor.LoggingFSM]], can be obtained by calling `getLog`.
|
||||
*/
|
||||
case class LogEntry[S, D](stateName: S, stateData: D, event: Any)
|
||||
|
||||
/**
|
||||
* This captures all of the managed state of the [[akka.actor.FSM]]: the state
|
||||
* name, the state data, possibly custom timeout, stop reason and replies
|
||||
* accumulated while processing the last message.
|
||||
*/
|
||||
case class State[S, D](stateName: S, stateData: D, timeout: Option[Duration] = None, stopReason: Option[Reason] = None, replies: List[Any] = Nil) {
|
||||
|
||||
/**
|
||||
|
|
@ -86,6 +150,9 @@ object FSM {
|
|||
copy(stateData = nextStateDate)
|
||||
}
|
||||
|
||||
/**
|
||||
* Internal API.
|
||||
*/
|
||||
private[akka] def withStopReason(reason: Reason): State[S, D] = {
|
||||
copy(stopReason = Some(reason))
|
||||
}
|
||||
|
|
@ -182,8 +249,19 @@ trait FSM[S, D] extends Listeners with ActorLogging {
|
|||
type Timeout = Option[Duration]
|
||||
type TransitionHandler = PartialFunction[(S, S), Unit]
|
||||
|
||||
// “import” so that it is visible without an import
|
||||
/*
|
||||
* “import” so that these are visible without an import
|
||||
*/
|
||||
|
||||
/**
|
||||
* This extractor is just convenience for matching a (S, S) pair, including a
|
||||
* reminder what the new state is.
|
||||
*/
|
||||
val -> = FSM.->
|
||||
|
||||
/**
|
||||
* This case object is received in case of a state timeout.
|
||||
*/
|
||||
val StateTimeout = FSM.StateTimeout
|
||||
|
||||
/**
|
||||
|
|
@ -202,13 +280,9 @@ trait FSM[S, D] extends Listeners with ActorLogging {
|
|||
* @param stateTimeout default state timeout for this state
|
||||
* @param stateFunction partial function describing response to input
|
||||
*/
|
||||
protected final def when(stateName: S, stateTimeout: Duration = null)(stateFunction: StateFunction): Unit =
|
||||
final def when(stateName: S, stateTimeout: Duration = null)(stateFunction: StateFunction): Unit =
|
||||
register(stateName, stateFunction, Option(stateTimeout))
|
||||
|
||||
@deprecated("use the more import-friendly variant taking a Duration", "2.0")
|
||||
protected final def when(stateName: S, stateTimeout: Timeout)(stateFunction: StateFunction): Unit =
|
||||
register(stateName, stateFunction, stateTimeout)
|
||||
|
||||
/**
|
||||
* Set initial state. Call this method from the constructor before the #initialize method.
|
||||
*
|
||||
|
|
@ -216,9 +290,7 @@ trait FSM[S, D] extends Listeners with ActorLogging {
|
|||
* @param stateData initial state data
|
||||
* @param timeout state timeout for the initial state, overriding the default timeout for that state
|
||||
*/
|
||||
protected final def startWith(stateName: S,
|
||||
stateData: D,
|
||||
timeout: Timeout = None): Unit =
|
||||
final def startWith(stateName: S, stateData: D, timeout: Timeout = None): Unit =
|
||||
currentState = FSM.State(stateName, stateData, timeout)
|
||||
|
||||
/**
|
||||
|
|
@ -228,7 +300,7 @@ trait FSM[S, D] extends Listeners with ActorLogging {
|
|||
* @param nextStateName state designator for the next state
|
||||
* @return state transition descriptor
|
||||
*/
|
||||
protected final def goto(nextStateName: S): State = FSM.State(nextStateName, currentState.stateData)
|
||||
final def goto(nextStateName: S): State = FSM.State(nextStateName, currentState.stateData)
|
||||
|
||||
/**
|
||||
* Produce "empty" transition descriptor. Return this from a state function
|
||||
|
|
@ -236,29 +308,29 @@ trait FSM[S, D] extends Listeners with ActorLogging {
|
|||
*
|
||||
* @return descriptor for staying in current state
|
||||
*/
|
||||
protected final def stay(): State = goto(currentState.stateName) // cannot directly use currentState because of the timeout field
|
||||
final def stay(): State = goto(currentState.stateName) // cannot directly use currentState because of the timeout field
|
||||
|
||||
/**
|
||||
* Produce change descriptor to stop this FSM actor with reason "Normal".
|
||||
*/
|
||||
protected final def stop(): State = stop(Normal)
|
||||
final def stop(): State = stop(Normal)
|
||||
|
||||
/**
|
||||
* Produce change descriptor to stop this FSM actor including specified reason.
|
||||
*/
|
||||
protected final def stop(reason: Reason): State = stop(reason, currentState.stateData)
|
||||
final def stop(reason: Reason): State = stop(reason, currentState.stateData)
|
||||
|
||||
/**
|
||||
* Produce change descriptor to stop this FSM actor including specified reason.
|
||||
*/
|
||||
protected final def stop(reason: Reason, stateData: D): State = stay using stateData withStopReason (reason)
|
||||
final def stop(reason: Reason, stateData: D): State = stay using stateData withStopReason (reason)
|
||||
|
||||
protected final class TransformHelper(func: StateFunction) {
|
||||
final class TransformHelper(func: StateFunction) {
|
||||
def using(andThen: PartialFunction[State, State]): StateFunction =
|
||||
func andThen (andThen orElse { case x ⇒ x })
|
||||
}
|
||||
|
||||
protected final def transform(func: StateFunction): TransformHelper = new TransformHelper(func)
|
||||
final def transform(func: StateFunction): TransformHelper = new TransformHelper(func)
|
||||
|
||||
/**
|
||||
* Schedule named timer to deliver message after given delay, possibly repeating.
|
||||
|
|
@ -268,7 +340,9 @@ trait FSM[S, D] extends Listeners with ActorLogging {
|
|||
* @param repeat send once if false, scheduleAtFixedRate if true
|
||||
* @return current state descriptor
|
||||
*/
|
||||
protected[akka] def setTimer(name: String, msg: Any, timeout: Duration, repeat: Boolean): State = {
|
||||
final def setTimer(name: String, msg: Any, timeout: Duration, repeat: Boolean): State = {
|
||||
if (debugEvent)
|
||||
log.debug("setting " + (if (repeat) "repeating " else "") + "timer '" + name + "'/" + timeout + ": " + msg)
|
||||
if (timers contains name) {
|
||||
timers(name).cancel
|
||||
}
|
||||
|
|
@ -282,24 +356,27 @@ trait FSM[S, D] extends Listeners with ActorLogging {
|
|||
* Cancel named timer, ensuring that the message is not subsequently delivered (no race).
|
||||
* @param name of the timer to cancel
|
||||
*/
|
||||
protected[akka] def cancelTimer(name: String): Unit =
|
||||
final def cancelTimer(name: String): Unit = {
|
||||
if (debugEvent)
|
||||
log.debug("canceling timer '" + name + "'")
|
||||
if (timers contains name) {
|
||||
timers(name).cancel
|
||||
timers -= name
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Inquire whether the named timer is still active. Returns true unless the
|
||||
* timer does not exist, has previously been canceled or if it was a
|
||||
* single-shot timer whose message was already received.
|
||||
*/
|
||||
protected[akka] final def timerActive_?(name: String) = timers contains name
|
||||
final def timerActive_?(name: String) = timers contains name
|
||||
|
||||
/**
|
||||
* Set state timeout explicitly. This method can safely be used from within a
|
||||
* state handler.
|
||||
*/
|
||||
protected final def setStateTimeout(state: S, timeout: Timeout): Unit = stateTimeouts(state) = timeout
|
||||
final def setStateTimeout(state: S, timeout: Timeout): Unit = stateTimeouts(state) = timeout
|
||||
|
||||
/**
|
||||
* Set handler which is called upon each state transition, i.e. not when
|
||||
|
|
@ -326,50 +403,52 @@ trait FSM[S, D] extends Listeners with ActorLogging {
|
|||
* <b>Multiple handlers may be installed, and every one of them will be
|
||||
* called, not only the first one matching.</b>
|
||||
*/
|
||||
protected final def onTransition(transitionHandler: TransitionHandler): Unit = transitionEvent :+= transitionHandler
|
||||
final def onTransition(transitionHandler: TransitionHandler): Unit = transitionEvent :+= transitionHandler
|
||||
|
||||
/**
|
||||
* Convenience wrapper for using a total function instead of a partial
|
||||
* function literal. To be used with onTransition.
|
||||
*/
|
||||
implicit protected final def total2pf(transitionHandler: (S, S) ⇒ Unit): TransitionHandler =
|
||||
implicit final def total2pf(transitionHandler: (S, S) ⇒ Unit): TransitionHandler =
|
||||
new TransitionHandler {
|
||||
def isDefinedAt(in: (S, S)) = true
|
||||
def apply(in: (S, S)) { transitionHandler(in._1, in._2) }
|
||||
}
|
||||
|
||||
/**
|
||||
* Set handler which is called upon termination of this FSM actor.
|
||||
* Set handler which is called upon termination of this FSM actor. Calling
|
||||
* this method again will overwrite the previous contents.
|
||||
*/
|
||||
protected final def onTermination(terminationHandler: PartialFunction[StopEvent, Unit]): Unit =
|
||||
final def onTermination(terminationHandler: PartialFunction[StopEvent, Unit]): Unit =
|
||||
terminateEvent = terminationHandler
|
||||
|
||||
/**
|
||||
* Set handler which is called upon reception of unhandled messages.
|
||||
* Set handler which is called upon reception of unhandled messages. Calling
|
||||
* this method again will overwrite the previous contents.
|
||||
*/
|
||||
protected final def whenUnhandled(stateFunction: StateFunction): Unit =
|
||||
final def whenUnhandled(stateFunction: StateFunction): Unit =
|
||||
handleEvent = stateFunction orElse handleEventDefault
|
||||
|
||||
/**
|
||||
* Verify existence of initial state and setup timers. This should be the
|
||||
* last call within the constructor.
|
||||
*/
|
||||
protected final def initialize: Unit = makeTransition(currentState)
|
||||
final def initialize: Unit = makeTransition(currentState)
|
||||
|
||||
/**
|
||||
* Return current state name (i.e. object of type S)
|
||||
*/
|
||||
protected[akka] def stateName: S = currentState.stateName
|
||||
final def stateName: S = currentState.stateName
|
||||
|
||||
/**
|
||||
* Return current state data (i.e. object of type D)
|
||||
*/
|
||||
protected[akka] def stateData: D = currentState.stateData
|
||||
final def stateData: D = currentState.stateData
|
||||
|
||||
/**
|
||||
* Return next state data (available in onTransition handlers)
|
||||
*/
|
||||
protected[akka] def nextStateData = nextState.stateData
|
||||
final def nextStateData = nextState.stateData
|
||||
|
||||
/*
|
||||
* ****************************************************************
|
||||
|
|
@ -377,6 +456,8 @@ trait FSM[S, D] extends Listeners with ActorLogging {
|
|||
* ****************************************************************
|
||||
*/
|
||||
|
||||
private[akka] def debugEvent: Boolean = false
|
||||
|
||||
/*
|
||||
* FSM State data and current timeout handling
|
||||
*/
|
||||
|
|
@ -436,7 +517,7 @@ trait FSM[S, D] extends Listeners with ActorLogging {
|
|||
* Main actor receive() method
|
||||
* *******************************************
|
||||
*/
|
||||
override final protected def receive: Receive = {
|
||||
override final def receive: Receive = {
|
||||
case TimeoutMarker(gen) ⇒
|
||||
if (generation == gen) {
|
||||
processMsg(StateTimeout, "state timeout")
|
||||
|
|
@ -524,7 +605,21 @@ trait FSM[S, D] extends Listeners with ActorLogging {
|
|||
}
|
||||
}
|
||||
|
||||
override def postStop(): Unit = { terminate(stay withStopReason Shutdown) }
|
||||
/**
|
||||
* Call `onTermination` hook; if you want to retain this behavior when
|
||||
* overriding make sure to call `super.postStop()`.
|
||||
*
|
||||
* Please note that this method is called by default from `preRestart()`,
|
||||
* so override that one if `onTermination` shall not be called during
|
||||
* restart.
|
||||
*/
|
||||
override def postStop(): Unit = {
|
||||
/*
|
||||
* setting this instance’s state to terminated does no harm during restart
|
||||
* since the new instance will initialize fresh using startWith()
|
||||
*/
|
||||
terminate(stay withStopReason Shutdown)
|
||||
}
|
||||
|
||||
private def terminate(nextState: State): Unit = {
|
||||
if (!currentState.stopReason.isDefined) {
|
||||
|
|
@ -541,13 +636,22 @@ trait FSM[S, D] extends Listeners with ActorLogging {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* All messages sent to the [[akka.actor.FSM]] will be wrapped inside an
|
||||
* `Event`, which allows pattern matching to extract both state and data.
|
||||
*/
|
||||
case class Event(event: Any, stateData: D)
|
||||
|
||||
/**
|
||||
* Case class representing the state of the [[akka.actor.FSM]] whithin the
|
||||
* `onTermination` block.
|
||||
*/
|
||||
case class StopEvent(reason: Reason, currentState: S, stateData: D)
|
||||
}
|
||||
|
||||
/**
|
||||
* Stackable trait for FSM which adds a rolling event log.
|
||||
* Stackable trait for [[akka.actor.FSM]] which adds a rolling event log and
|
||||
* debug logging capabilities (analogous to [[akka.event.LoggingReceive]]).
|
||||
*
|
||||
* @since 1.2
|
||||
*/
|
||||
|
|
@ -557,7 +661,7 @@ trait LoggingFSM[S, D] extends FSM[S, D] { this: Actor ⇒
|
|||
|
||||
def logDepth: Int = 0
|
||||
|
||||
private val debugEvent = context.system.settings.FsmDebugEvent
|
||||
private[akka] override val debugEvent = context.system.settings.FsmDebugEvent
|
||||
|
||||
private val events = new Array[Event](logDepth)
|
||||
private val states = new Array[AnyRef](logDepth)
|
||||
|
|
@ -574,18 +678,6 @@ trait LoggingFSM[S, D] extends FSM[S, D] { this: Actor ⇒
|
|||
}
|
||||
}
|
||||
|
||||
protected[akka] abstract override def setTimer(name: String, msg: Any, timeout: Duration, repeat: Boolean): State = {
|
||||
if (debugEvent)
|
||||
log.debug("setting " + (if (repeat) "repeating " else "") + "timer '" + name + "'/" + timeout + ": " + msg)
|
||||
super.setTimer(name, msg, timeout, repeat)
|
||||
}
|
||||
|
||||
protected[akka] abstract override def cancelTimer(name: String): Unit = {
|
||||
if (debugEvent)
|
||||
log.debug("canceling timer '" + name + "'")
|
||||
super.cancelTimer(name)
|
||||
}
|
||||
|
||||
private[akka] abstract override def processEvent(event: Event, source: AnyRef): Unit = {
|
||||
if (debugEvent) {
|
||||
val srcstr = source match {
|
||||
|
|
@ -614,6 +706,7 @@ trait LoggingFSM[S, D] extends FSM[S, D] { this: Actor ⇒
|
|||
/**
|
||||
* Retrieve current rolling log in oldest-first order. The log is filled with
|
||||
* each incoming event before processing by the user supplied state handler.
|
||||
* The log entries are lost when this actor is restarted.
|
||||
*/
|
||||
protected def getLog: IndexedSeq[LogEntry[S, D]] = {
|
||||
val log = events zip states filter (_._1 ne null) map (x ⇒ LogEntry(x._2.asInstanceOf[S], x._1.stateData, x._1.event))
|
||||
|
|
|
|||
|
|
@ -9,8 +9,13 @@ import scala.collection.JavaConversions._
|
|||
import java.lang.{ Iterable ⇒ JIterable }
|
||||
import akka.util.Duration
|
||||
|
||||
/**
|
||||
* ChildRestartStats is the statistics kept by every parent Actor for every child Actor
|
||||
* and is used for SupervisorStrategies to know how to deal with problems that occur for the children.
|
||||
*/
|
||||
case class ChildRestartStats(val child: ActorRef, var maxNrOfRetriesCount: Int = 0, var restartTimeWindowStartNanos: Long = 0L) {
|
||||
|
||||
//FIXME How about making ChildRestartStats immutable and then move these methods into the actual supervisor strategies?
|
||||
def requestRestartPermission(retriesWindow: (Option[Int], Option[Int])): Boolean =
|
||||
retriesWindow match {
|
||||
case (Some(retries), _) if retries < 1 ⇒ false
|
||||
|
|
@ -160,20 +165,21 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits {
|
|||
def makeDecider(flat: Iterable[CauseDirective]): Decider = {
|
||||
val directives = sort(flat)
|
||||
|
||||
{
|
||||
case x ⇒ directives find (_._1 isInstance x) map (_._2) getOrElse Escalate
|
||||
}
|
||||
{ case x ⇒ directives collectFirst { case (c, d) if c isInstance x ⇒ d } getOrElse Escalate }
|
||||
}
|
||||
|
||||
def makeDecider(func: JDecider): Decider = {
|
||||
case x ⇒ func(x)
|
||||
}
|
||||
/**
|
||||
* Converts a Java Decider into a Scala Decider
|
||||
*/
|
||||
def makeDecider(func: JDecider): Decider = { case x ⇒ func(x) }
|
||||
|
||||
/**
|
||||
* Sort so that subtypes always precede their supertypes, but without
|
||||
* obeying any order between unrelated subtypes (insert sort).
|
||||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
def sort(in: Iterable[CauseDirective]): Seq[CauseDirective] =
|
||||
private[akka] def sort(in: Iterable[CauseDirective]): Seq[CauseDirective] =
|
||||
(new ArrayBuffer[CauseDirective](in.size) /: in) { (buf, ca) ⇒
|
||||
buf.indexWhere(_._1 isAssignableFrom ca._1) match {
|
||||
case -1 ⇒ buf append ca
|
||||
|
|
@ -184,14 +190,21 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits {
|
|||
|
||||
private[akka] def withinTimeRangeOption(withinTimeRange: Duration): Option[Duration] =
|
||||
if (withinTimeRange.isFinite && withinTimeRange >= Duration.Zero) Some(withinTimeRange) else None
|
||||
|
||||
private[akka] def maxNrOfRetriesOption(maxNrOfRetries: Int): Option[Int] =
|
||||
if (maxNrOfRetries < 0) None else Some(maxNrOfRetries)
|
||||
}
|
||||
|
||||
/**
|
||||
* An Akka SupervisorStrategy is the policy to apply for crashing children
|
||||
*/
|
||||
abstract class SupervisorStrategy {
|
||||
|
||||
import SupervisorStrategy._
|
||||
|
||||
/**
|
||||
* Returns the Decider that is associated with this SupervisorStrategy
|
||||
*/
|
||||
def decider: Decider
|
||||
|
||||
/**
|
||||
|
|
@ -204,21 +217,19 @@ abstract class SupervisorStrategy {
|
|||
*/
|
||||
def processFailure(context: ActorContext, restart: Boolean, child: ActorRef, cause: Throwable, stats: ChildRestartStats, children: Iterable[ChildRestartStats]): Unit
|
||||
|
||||
def handleSupervisorFailing(supervisor: ActorRef, children: Iterable[ActorRef]): Unit = {
|
||||
if (children.nonEmpty)
|
||||
children.foreach(_.asInstanceOf[InternalActorRef].suspend())
|
||||
}
|
||||
//FIXME docs
|
||||
def handleSupervisorFailing(supervisor: ActorRef, children: Iterable[ActorRef]): Unit =
|
||||
if (children.nonEmpty) children.foreach(_.asInstanceOf[InternalActorRef].suspend())
|
||||
|
||||
def handleSupervisorRestarted(cause: Throwable, supervisor: ActorRef, children: Iterable[ActorRef]): Unit = {
|
||||
if (children.nonEmpty)
|
||||
children.foreach(_.asInstanceOf[InternalActorRef].restart(cause))
|
||||
}
|
||||
//FIXME docs
|
||||
def handleSupervisorRestarted(cause: Throwable, supervisor: ActorRef, children: Iterable[ActorRef]): Unit =
|
||||
if (children.nonEmpty) children.foreach(_.asInstanceOf[InternalActorRef].restart(cause))
|
||||
|
||||
/**
|
||||
* Returns whether it processed the failure or not
|
||||
*/
|
||||
def handleFailure(context: ActorContext, child: ActorRef, cause: Throwable, stats: ChildRestartStats, children: Iterable[ChildRestartStats]): Boolean = {
|
||||
val directive = if (decider.isDefinedAt(cause)) decider(cause) else Escalate
|
||||
val directive = if (decider.isDefinedAt(cause)) decider(cause) else Escalate //FIXME applyOrElse in Scala 2.10
|
||||
directive match {
|
||||
case Resume ⇒ child.asInstanceOf[InternalActorRef].resume(); true
|
||||
case Restart ⇒ processFailure(context, true, child, cause, stats, children); true
|
||||
|
|
@ -242,6 +253,8 @@ abstract class SupervisorStrategy {
|
|||
case class AllForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration = Duration.Inf)(val decider: SupervisorStrategy.Decider)
|
||||
extends SupervisorStrategy {
|
||||
|
||||
import SupervisorStrategy._
|
||||
|
||||
def this(maxNrOfRetries: Int, withinTimeRange: Duration, decider: SupervisorStrategy.JDecider) =
|
||||
this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(decider))
|
||||
|
||||
|
|
@ -256,9 +269,7 @@ case class AllForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration
|
|||
* every call to requestRestartPermission, assuming that strategies are shared
|
||||
* across actors and thus this field does not take up much space
|
||||
*/
|
||||
private val retriesWindow = (
|
||||
SupervisorStrategy.maxNrOfRetriesOption(maxNrOfRetries),
|
||||
SupervisorStrategy.withinTimeRangeOption(withinTimeRange).map(_.toMillis.toInt))
|
||||
private val retriesWindow = (maxNrOfRetriesOption(maxNrOfRetries), withinTimeRangeOption(withinTimeRange).map(_.toMillis.toInt))
|
||||
|
||||
def handleChildTerminated(context: ActorContext, child: ActorRef, children: Iterable[ActorRef]): Unit = {}
|
||||
|
||||
|
|
|
|||
|
|
@ -21,8 +21,7 @@ import java.nio.channels.{
|
|||
import scala.collection.mutable
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.generic.CanBuildFrom
|
||||
import com.eaio.uuid.UUID
|
||||
|
||||
import java.util.UUID
|
||||
/**
|
||||
* IO messages and iteratees.
|
||||
*
|
||||
|
|
@ -31,7 +30,7 @@ import com.eaio.uuid.UUID
|
|||
*/
|
||||
object IO {
|
||||
|
||||
final class DivergentIterateeException extends Exception("Iteratees should not return a continuation when receiving EOF")
|
||||
final class DivergentIterateeException extends IllegalStateException("Iteratees should not return a continuation when receiving EOF")
|
||||
|
||||
/**
|
||||
* An immutable handle to a Java NIO Channel. Contains a reference to the
|
||||
|
|
@ -65,14 +64,14 @@ object IO {
|
|||
* A [[akka.actor.IO.Handle]] to a ReadableByteChannel.
|
||||
*/
|
||||
sealed trait ReadHandle extends Handle with Product {
|
||||
override def asReadable = this
|
||||
override def asReadable: ReadHandle = this
|
||||
}
|
||||
|
||||
/**
|
||||
* A [[akka.actor.IO.Handle]] to a WritableByteChannel.
|
||||
*/
|
||||
sealed trait WriteHandle extends Handle with Product {
|
||||
override def asWritable = this
|
||||
override def asWritable: WriteHandle = this
|
||||
|
||||
/**
|
||||
* Sends a request to the [[akka.actor.IOManager]] to write to the
|
||||
|
|
@ -89,16 +88,16 @@ object IO {
|
|||
* created by [[akka.actor.IOManager]].connect() and
|
||||
* [[akka.actor.IO.ServerHandle]].accept().
|
||||
*/
|
||||
case class SocketHandle(owner: ActorRef, ioManager: ActorRef, uuid: UUID = new UUID()) extends ReadHandle with WriteHandle {
|
||||
override def asSocket = this
|
||||
case class SocketHandle(owner: ActorRef, ioManager: ActorRef, uuid: UUID = UUID.randomUUID()) extends ReadHandle with WriteHandle {
|
||||
override def asSocket: SocketHandle = this
|
||||
}
|
||||
|
||||
/**
|
||||
* A [[akka.actor.IO.Handle]] to a ServerSocketChannel. Instances are
|
||||
* normally created by [[akka.actor.IOManager]].listen().
|
||||
*/
|
||||
case class ServerHandle(owner: ActorRef, ioManager: ActorRef, uuid: UUID = new UUID()) extends Handle {
|
||||
override def asServer = this
|
||||
case class ServerHandle(owner: ActorRef, ioManager: ActorRef, uuid: UUID = UUID.randomUUID()) extends Handle {
|
||||
override def asServer: ServerHandle = this
|
||||
|
||||
/**
|
||||
* Sends a request to the [[akka.actor.IOManager]] to accept an incoming
|
||||
|
|
@ -321,16 +320,18 @@ object IO {
|
|||
}
|
||||
|
||||
object Chunk {
|
||||
val empty = Chunk(ByteString.empty)
|
||||
val empty: Chunk = new Chunk(ByteString.empty)
|
||||
}
|
||||
|
||||
/**
|
||||
* Part of an [[akka.actor.IO.Input]] stream that contains a chunk of bytes.
|
||||
*/
|
||||
case class Chunk(bytes: ByteString) extends Input {
|
||||
def ++(that: Input) = that match {
|
||||
case Chunk(more) ⇒ Chunk(bytes ++ more)
|
||||
case _: EOF ⇒ that
|
||||
final override def ++(that: Input): Input = that match {
|
||||
case Chunk(more) if more.isEmpty ⇒ this
|
||||
case c: Chunk if bytes.isEmpty ⇒ c
|
||||
case Chunk(more) ⇒ Chunk(bytes ++ more)
|
||||
case _: EOF ⇒ that
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -343,7 +344,7 @@ object IO {
|
|||
* Iteratee.recover() in order to handle it properly.
|
||||
*/
|
||||
case class EOF(cause: Option[Exception]) extends Input {
|
||||
def ++(that: Input) = that
|
||||
final override def ++(that: Input): Input = that
|
||||
}
|
||||
|
||||
object Iteratee {
|
||||
|
|
@ -353,7 +354,15 @@ object IO {
|
|||
* inferred as an Iteratee and not as a Done.
|
||||
*/
|
||||
def apply[A](value: A): Iteratee[A] = Done(value)
|
||||
|
||||
/**
|
||||
* Returns Iteratee.unit
|
||||
*/
|
||||
def apply(): Iteratee[Unit] = unit
|
||||
|
||||
/**
|
||||
* The single value representing Done(())
|
||||
*/
|
||||
val unit: Iteratee[Unit] = Done(())
|
||||
}
|
||||
|
||||
|
|
@ -446,6 +455,7 @@ object IO {
|
|||
*/
|
||||
final case class Cont[+A](f: Input ⇒ (Iteratee[A], Input), error: Option[Exception] = None) extends Iteratee[A]
|
||||
|
||||
//FIXME general description of what an IterateeRef is and how it is used, potentially with link to docs
|
||||
object IterateeRef {
|
||||
|
||||
/**
|
||||
|
|
@ -478,13 +488,14 @@ object IO {
|
|||
* 'refFactory' is used to provide the default value for new keys.
|
||||
*/
|
||||
class Map[K, V] private (refFactory: ⇒ IterateeRef[V], underlying: mutable.Map[K, IterateeRef[V]] = mutable.Map.empty[K, IterateeRef[V]]) extends mutable.Map[K, IterateeRef[V]] {
|
||||
def get(key: K) = Some(underlying.getOrElseUpdate(key, refFactory))
|
||||
def iterator = underlying.iterator
|
||||
def +=(kv: (K, IterateeRef[V])) = { underlying += kv; this }
|
||||
def -=(key: K) = { underlying -= key; this }
|
||||
override def get(key: K) = Some(underlying.getOrElseUpdate(key, refFactory))
|
||||
override def iterator = underlying.iterator
|
||||
override def +=(kv: (K, IterateeRef[V])) = { underlying += kv; this }
|
||||
override def -=(key: K) = { underlying -= key; this }
|
||||
override def empty = new Map[K, V](refFactory)
|
||||
}
|
||||
|
||||
//FIXME general description of what an Map is and how it is used, potentially with link to docs
|
||||
object Map {
|
||||
/**
|
||||
* Uses a factory to create the initial IterateeRef for each new key.
|
||||
|
|
@ -501,7 +512,6 @@ object IO {
|
|||
*/
|
||||
def async[K]()(implicit executor: ExecutionContext): IterateeRef.Map[K, Unit] = new Map(IterateeRef.async())
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -511,8 +521,11 @@ object IO {
|
|||
* for details.
|
||||
*/
|
||||
trait IterateeRef[A] {
|
||||
//FIXME Add docs
|
||||
def flatMap(f: A ⇒ Iteratee[A]): Unit
|
||||
//FIXME Add docs
|
||||
def map(f: A ⇒ A): Unit
|
||||
//FIXME Add docs
|
||||
def apply(input: Input): Unit
|
||||
}
|
||||
|
||||
|
|
@ -529,12 +542,16 @@ object IO {
|
|||
*/
|
||||
final class IterateeRefSync[A](initial: Iteratee[A]) extends IterateeRef[A] {
|
||||
private var _value: (Iteratee[A], Input) = (initial, Chunk.empty)
|
||||
def flatMap(f: A ⇒ Iteratee[A]): Unit = _value = _value match {
|
||||
override def flatMap(f: A ⇒ Iteratee[A]): Unit = _value = _value match {
|
||||
case (iter, chunk @ Chunk(bytes)) if bytes.nonEmpty ⇒ (iter flatMap f)(chunk)
|
||||
case (iter, input) ⇒ (iter flatMap f, input)
|
||||
}
|
||||
def map(f: A ⇒ A): Unit = _value = (_value._1 map f, _value._2)
|
||||
def apply(input: Input): Unit = _value = _value._1(_value._2 ++ input)
|
||||
override def map(f: A ⇒ A): Unit = _value = (_value._1 map f, _value._2)
|
||||
override def apply(input: Input): Unit = _value = _value._1(_value._2 ++ input)
|
||||
|
||||
/**
|
||||
* Returns the current value of this IterateeRefSync
|
||||
*/
|
||||
def value: (Iteratee[A], Input) = _value
|
||||
}
|
||||
|
||||
|
|
@ -554,12 +571,16 @@ object IO {
|
|||
*/
|
||||
final class IterateeRefAsync[A](initial: Iteratee[A])(implicit executor: ExecutionContext) extends IterateeRef[A] {
|
||||
private var _value: Future[(Iteratee[A], Input)] = Future((initial, Chunk.empty))
|
||||
def flatMap(f: A ⇒ Iteratee[A]): Unit = _value = _value map {
|
||||
override def flatMap(f: A ⇒ Iteratee[A]): Unit = _value = _value map {
|
||||
case (iter, chunk @ Chunk(bytes)) if bytes.nonEmpty ⇒ (iter flatMap f)(chunk)
|
||||
case (iter, input) ⇒ (iter flatMap f, input)
|
||||
}
|
||||
def map(f: A ⇒ A): Unit = _value = _value map (v ⇒ (v._1 map f, v._2))
|
||||
def apply(input: Input): Unit = _value = _value map (v ⇒ v._1(v._2 ++ input))
|
||||
override def map(f: A ⇒ A): Unit = _value = _value map (v ⇒ (v._1 map f, v._2))
|
||||
override def apply(input: Input): Unit = _value = _value map (v ⇒ v._1(v._2 ++ input))
|
||||
|
||||
/**
|
||||
* Returns a Future which will hold the future value of this IterateeRefAsync
|
||||
*/
|
||||
def future: Future[(Iteratee[A], Input)] = _value
|
||||
}
|
||||
|
||||
|
|
@ -703,10 +724,9 @@ object IO {
|
|||
/**
|
||||
* An Iteratee that continually repeats an Iteratee.
|
||||
*
|
||||
* TODO: Should terminate on EOF
|
||||
* FIXME TODO: Should terminate on EOF
|
||||
*/
|
||||
def repeat(iter: Iteratee[Unit]): Iteratee[Unit] =
|
||||
iter flatMap (_ ⇒ repeat(iter))
|
||||
def repeat(iter: Iteratee[Unit]): Iteratee[Unit] = iter flatMap (_ ⇒ repeat(iter))
|
||||
|
||||
/**
|
||||
* An Iteratee that applies an Iteratee to each element of a Traversable
|
||||
|
|
@ -781,7 +801,7 @@ object IO {
|
|||
* An IOManager does not need to be manually stopped when not in use as it will
|
||||
* automatically enter an idle state when it has no channels to manage.
|
||||
*/
|
||||
final class IOManager private (system: ActorSystem) extends Extension {
|
||||
final class IOManager private (system: ActorSystem) extends Extension { //FIXME how about taking an ActorContext
|
||||
/**
|
||||
* A reference to the [[akka.actor.IOManagerActor]] that performs the actual
|
||||
* IO. It communicates with other actors using subclasses of
|
||||
|
|
@ -862,9 +882,10 @@ final class IOManager private (system: ActorSystem) extends Extension {
|
|||
|
||||
}
|
||||
|
||||
//FIXME add docs
|
||||
object IOManager extends ExtensionId[IOManager] with ExtensionIdProvider {
|
||||
override def lookup = this
|
||||
override def createExtension(system: ExtendedActorSystem) = new IOManager(system)
|
||||
override def lookup: IOManager.type = this
|
||||
override def createExtension(system: ExtendedActorSystem): IOManager = new IOManager(system)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -875,7 +896,7 @@ object IOManager extends ExtensionId[IOManager] with ExtensionIdProvider {
|
|||
final class IOManagerActor extends Actor with ActorLogging {
|
||||
import SelectionKey.{ OP_READ, OP_WRITE, OP_ACCEPT, OP_CONNECT }
|
||||
|
||||
private val bufferSize = 8192 // TODO: make buffer size configurable
|
||||
private val bufferSize = 8192 // FIXME TODO: make configurable
|
||||
|
||||
private type ReadChannel = ReadableByteChannel with SelectableChannel
|
||||
private type WriteChannel = WritableByteChannel with SelectableChannel
|
||||
|
|
@ -898,7 +919,7 @@ final class IOManagerActor extends Actor with ActorLogging {
|
|||
private var lastSelect = 0
|
||||
|
||||
/** force a select when lastSelect reaches this amount */
|
||||
private val selectAt = 100
|
||||
private val selectAt = 100 // FIXME TODO: make configurable
|
||||
|
||||
/** true while the selector is open and channels.nonEmpty */
|
||||
private var running = false
|
||||
|
|
@ -948,9 +969,7 @@ final class IOManagerActor extends Actor with ActorLogging {
|
|||
lastSelect = 0
|
||||
}
|
||||
|
||||
private def forwardFailure(f: ⇒ Unit): Unit = {
|
||||
try { f } catch { case NonFatal(e) ⇒ sender ! Status.Failure(e) }
|
||||
}
|
||||
private def forwardFailure(f: ⇒ Unit): Unit = try f catch { case NonFatal(e) ⇒ sender ! Status.Failure(e) }
|
||||
|
||||
private def setSocketOptions(socket: java.net.Socket, options: Seq[IO.SocketOption]) {
|
||||
options foreach {
|
||||
|
|
@ -968,7 +987,7 @@ final class IOManagerActor extends Actor with ActorLogging {
|
|||
}
|
||||
}
|
||||
|
||||
protected def receive = {
|
||||
def receive = {
|
||||
case Select ⇒
|
||||
select()
|
||||
if (running) self ! Select
|
||||
|
|
@ -986,7 +1005,7 @@ final class IOManagerActor extends Actor with ActorLogging {
|
|||
forwardFailure(sock.setPerformancePreferences(connTime, latency, bandwidth))
|
||||
}
|
||||
|
||||
channel.socket bind (address, 1000) // TODO: make backlog configurable
|
||||
channel.socket bind (address, 1000) // FIXME TODO: make backlog configurable
|
||||
channels update (server, channel)
|
||||
channel register (selector, OP_ACCEPT, server)
|
||||
server.owner ! IO.Listening(server, channel.socket.getLocalSocketAddress())
|
||||
|
|
@ -1049,29 +1068,13 @@ final class IOManagerActor extends Actor with ActorLogging {
|
|||
private def process(key: SelectionKey) {
|
||||
val handle = key.attachment.asInstanceOf[IO.Handle]
|
||||
try {
|
||||
if (key.isConnectable) key.channel match {
|
||||
case channel: SocketChannel ⇒ connect(handle.asSocket, channel)
|
||||
}
|
||||
if (key.isAcceptable) key.channel match {
|
||||
case channel: ServerSocketChannel ⇒ accept(handle.asServer, channel)
|
||||
}
|
||||
if (key.isReadable) key.channel match {
|
||||
case channel: ReadChannel ⇒ read(handle.asReadable, channel)
|
||||
}
|
||||
if (key.isWritable) key.channel match {
|
||||
case channel: WriteChannel ⇒
|
||||
try {
|
||||
write(handle.asWritable, channel)
|
||||
} catch {
|
||||
case e: IOException ⇒
|
||||
// ignore, let it fail on read to ensure nothing left in read buffer.
|
||||
}
|
||||
}
|
||||
if (key.isConnectable) key.channel match { case channel: SocketChannel ⇒ connect(handle.asSocket, channel) }
|
||||
if (key.isAcceptable) key.channel match { case channel: ServerSocketChannel ⇒ accept(handle.asServer, channel) }
|
||||
if (key.isReadable) key.channel match { case channel: ReadChannel ⇒ read(handle.asReadable, channel) }
|
||||
if (key.isWritable) key.channel match { case channel: WriteChannel ⇒ try write(handle.asWritable, channel) catch { case e: IOException ⇒ } } // ignore, let it fail on read to ensure nothing left in read buffer.
|
||||
} catch {
|
||||
case e: ClassCastException ⇒ cleanup(handle, Some(e))
|
||||
case e: CancelledKeyException ⇒ cleanup(handle, Some(e))
|
||||
case e: IOException ⇒ cleanup(handle, Some(e))
|
||||
case e: ActorInitializationException ⇒ cleanup(handle, Some(e))
|
||||
case e @ (_: ClassCastException | _: CancelledKeyException | _: IOException | _: ActorInitializationException) ⇒
|
||||
cleanup(handle, Some(e.asInstanceOf[Exception])) //Scala patmat is broken
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -1090,9 +1093,6 @@ final class IOManagerActor extends Actor with ActorLogging {
|
|||
}
|
||||
}
|
||||
|
||||
private def setOps(handle: IO.Handle, ops: Int): Unit =
|
||||
channels(handle) keyFor selector interestOps ops
|
||||
|
||||
private def addOps(handle: IO.Handle, ops: Int) {
|
||||
val key = channels(handle) keyFor selector
|
||||
val cur = key.interestOps
|
||||
|
|
@ -1158,9 +1158,9 @@ final class IOManagerActor extends Actor with ActorLogging {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
//FIXME is this public API?
|
||||
final class WriteBuffer(bufferSize: Int) {
|
||||
private val _queue = new java.util.ArrayDeque[ByteString]
|
||||
private val _buffer = ByteBuffer.allocate(bufferSize)
|
||||
|
|
@ -1182,9 +1182,9 @@ final class WriteBuffer(bufferSize: Int) {
|
|||
this
|
||||
}
|
||||
|
||||
def length = _length
|
||||
def length: Int = _length
|
||||
|
||||
def isEmpty = _length == 0
|
||||
def isEmpty: Boolean = _length == 0
|
||||
|
||||
def write(channel: WritableByteChannel with SelectableChannel): Int = {
|
||||
@tailrec
|
||||
|
|
|
|||
|
|
@ -18,12 +18,24 @@ import akka.routing._
|
|||
*/
|
||||
object Props {
|
||||
|
||||
/**
|
||||
* The defaultCreator, simply throws an UnsupportedOperationException when applied, which is used when creating a Props
|
||||
*/
|
||||
final val defaultCreator: () ⇒ Actor = () ⇒ throw new UnsupportedOperationException("No actor creator specified!")
|
||||
|
||||
/**
|
||||
* The defaultRoutedProps is NoRouter which is used when creating a Props
|
||||
*/
|
||||
final val defaultRoutedProps: RouterConfig = NoRouter
|
||||
|
||||
/**
|
||||
* The default Deploy instance which is used when creating a Props
|
||||
*/
|
||||
final val defaultDeploy = Deploy()
|
||||
|
||||
/**
|
||||
* A Props instance whose creator will create an actor that doesn't respond to any message
|
||||
*/
|
||||
final val empty = new Props(() ⇒ new Actor { def receive = Actor.emptyBehavior })
|
||||
|
||||
/**
|
||||
|
|
@ -49,8 +61,7 @@ object Props {
|
|||
* Returns a Props that has default values except for "creator" which will be a function that creates an instance
|
||||
* of the supplied class using the default constructor.
|
||||
*/
|
||||
def apply(actorClass: Class[_ <: Actor]): Props =
|
||||
default.withCreator(actorClass)
|
||||
def apply(actorClass: Class[_ <: Actor]): Props = default.withCreator(actorClass)
|
||||
|
||||
/**
|
||||
* Returns a Props that has default values except for "creator" which will be a function that creates an instance
|
||||
|
|
@ -58,18 +69,18 @@ object Props {
|
|||
*
|
||||
* Scala API.
|
||||
*/
|
||||
def apply(creator: ⇒ Actor): Props =
|
||||
default.withCreator(creator)
|
||||
def apply(creator: ⇒ Actor): Props = default.withCreator(creator)
|
||||
|
||||
/**
|
||||
* Returns a Props that has default values except for "creator" which will be a function that creates an instance
|
||||
* using the supplied thunk.
|
||||
*/
|
||||
def apply(creator: Creator[_ <: Actor]): Props =
|
||||
default.withCreator(creator.create)
|
||||
def apply(creator: Creator[_ <: Actor]): Props = default.withCreator(creator.create)
|
||||
|
||||
def apply(behavior: ActorContext ⇒ Actor.Receive): Props =
|
||||
apply(new Actor { def receive = behavior(context) })
|
||||
/**
|
||||
* Returns a new Props whose creator will instantiate an Actor that has the behavior specified
|
||||
*/
|
||||
def apply(behavior: ActorContext ⇒ Actor.Receive): Props = apply(new Actor { def receive = behavior(context) })
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -175,4 +186,4 @@ case class Props(
|
|||
*/
|
||||
private[akka] case class FromClassCreator(clazz: Class[_ <: Actor]) extends Function0[Actor] {
|
||||
def apply(): Actor = clazz.newInstance
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -5,10 +5,13 @@
|
|||
package akka.actor
|
||||
|
||||
import akka.util.Duration
|
||||
import org.jboss.netty.akka.util.{ TimerTask, HashedWheelTimer, Timeout ⇒ HWTimeout }
|
||||
import akka.util.internal.{ TimerTask, HashedWheelTimer, Timeout ⇒ HWTimeout, Timer }
|
||||
import akka.event.LoggingAdapter
|
||||
import akka.dispatch.MessageDispatcher
|
||||
import java.io.Closeable
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
import scala.annotation.tailrec
|
||||
import akka.util.internal._
|
||||
|
||||
//#scheduler
|
||||
/**
|
||||
|
|
@ -119,7 +122,7 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer,
|
|||
log: LoggingAdapter,
|
||||
dispatcher: ⇒ MessageDispatcher) extends Scheduler with Closeable {
|
||||
|
||||
def schedule(initialDelay: Duration, delay: Duration, receiver: ActorRef, message: Any): Cancellable = {
|
||||
override def schedule(initialDelay: Duration, delay: Duration, receiver: ActorRef, message: Any): Cancellable = {
|
||||
val continuousCancellable = new ContinuousCancellable
|
||||
continuousCancellable.init(
|
||||
hashedWheelTimer.newTimeout(
|
||||
|
|
@ -134,7 +137,7 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer,
|
|||
initialDelay))
|
||||
}
|
||||
|
||||
def schedule(initialDelay: Duration, delay: Duration)(f: ⇒ Unit): Cancellable = {
|
||||
override def schedule(initialDelay: Duration, delay: Duration)(f: ⇒ Unit): Cancellable = {
|
||||
val continuousCancellable = new ContinuousCancellable
|
||||
continuousCancellable.init(
|
||||
hashedWheelTimer.newTimeout(
|
||||
|
|
@ -148,7 +151,7 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer,
|
|||
initialDelay))
|
||||
}
|
||||
|
||||
def schedule(initialDelay: Duration, delay: Duration, runnable: Runnable): Cancellable = {
|
||||
override def schedule(initialDelay: Duration, delay: Duration, runnable: Runnable): Cancellable = {
|
||||
val continuousCancellable = new ContinuousCancellable
|
||||
continuousCancellable.init(
|
||||
hashedWheelTimer.newTimeout(
|
||||
|
|
@ -161,7 +164,7 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer,
|
|||
initialDelay))
|
||||
}
|
||||
|
||||
def scheduleOnce(delay: Duration, runnable: Runnable): Cancellable =
|
||||
override def scheduleOnce(delay: Duration, runnable: Runnable): Cancellable =
|
||||
new DefaultCancellable(
|
||||
hashedWheelTimer.newTimeout(
|
||||
new TimerTask() {
|
||||
|
|
@ -169,7 +172,7 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer,
|
|||
},
|
||||
delay))
|
||||
|
||||
def scheduleOnce(delay: Duration, receiver: ActorRef, message: Any): Cancellable =
|
||||
override def scheduleOnce(delay: Duration, receiver: ActorRef, message: Any): Cancellable =
|
||||
new DefaultCancellable(
|
||||
hashedWheelTimer.newTimeout(
|
||||
new TimerTask {
|
||||
|
|
@ -177,7 +180,7 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer,
|
|||
},
|
||||
delay))
|
||||
|
||||
def scheduleOnce(delay: Duration)(f: ⇒ Unit): Cancellable =
|
||||
override def scheduleOnce(delay: Duration)(f: ⇒ Unit): Cancellable =
|
||||
new DefaultCancellable(
|
||||
hashedWheelTimer.newTimeout(
|
||||
new TimerTask with Runnable {
|
||||
|
|
@ -188,11 +191,7 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer,
|
|||
|
||||
private trait ContinuousScheduling { this: TimerTask ⇒
|
||||
def scheduleNext(timeout: HWTimeout, delay: Duration, delegator: ContinuousCancellable) {
|
||||
try {
|
||||
delegator.swap(timeout.getTimer.newTimeout(this, delay))
|
||||
} catch {
|
||||
case _: IllegalStateException ⇒ // stop recurring if timer is stopped
|
||||
}
|
||||
try delegator.swap(timeout.getTimer.newTimeout(this, delay)) catch { case _: IllegalStateException ⇒ } // stop recurring if timer is stopped
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -203,54 +202,50 @@ class DefaultScheduler(hashedWheelTimer: HashedWheelTimer,
|
|||
}
|
||||
}
|
||||
|
||||
def close() = {
|
||||
override def close(): Unit = {
|
||||
import scala.collection.JavaConverters._
|
||||
hashedWheelTimer.stop().asScala foreach execDirectly
|
||||
}
|
||||
}
|
||||
|
||||
private[akka] object ContinuousCancellable {
|
||||
val initial: HWTimeout = new HWTimeout {
|
||||
override def getTimer: Timer = null
|
||||
override def getTask: TimerTask = null
|
||||
override def isExpired: Boolean = false
|
||||
override def isCancelled: Boolean = false
|
||||
override def cancel: Unit = ()
|
||||
}
|
||||
|
||||
val cancelled: HWTimeout = new HWTimeout {
|
||||
override def getTimer: Timer = null
|
||||
override def getTask: TimerTask = null
|
||||
override def isExpired: Boolean = false
|
||||
override def isCancelled: Boolean = true
|
||||
override def cancel: Unit = ()
|
||||
}
|
||||
}
|
||||
/**
|
||||
* Wrapper of a [[org.jboss.netty.akka.util.Timeout]] that delegates all
|
||||
* methods. Needed to be able to cancel continuous tasks,
|
||||
* since they create new Timeout for each tick.
|
||||
*/
|
||||
private[akka] class ContinuousCancellable extends Cancellable {
|
||||
@volatile
|
||||
private var delegate: HWTimeout = _
|
||||
@volatile
|
||||
private var cancelled = false
|
||||
|
||||
private[akka] class ContinuousCancellable extends AtomicReference[HWTimeout](ContinuousCancellable.initial) with Cancellable {
|
||||
private[akka] def init(initialTimeout: HWTimeout): this.type = {
|
||||
delegate = initialTimeout
|
||||
compareAndSet(ContinuousCancellable.initial, initialTimeout)
|
||||
this
|
||||
}
|
||||
|
||||
private[akka] def swap(newTimeout: HWTimeout): Unit = {
|
||||
val wasCancelled = isCancelled
|
||||
delegate = newTimeout
|
||||
if (wasCancelled || isCancelled) cancel()
|
||||
@tailrec private[akka] final def swap(newTimeout: HWTimeout): Unit = get match {
|
||||
case some if some.isCancelled ⇒ try cancel() finally newTimeout.cancel()
|
||||
case some ⇒ if (!compareAndSet(some, newTimeout)) swap(newTimeout)
|
||||
}
|
||||
|
||||
def isCancelled(): Boolean = {
|
||||
// delegate is initially null, but this object will not be exposed to the world until after init
|
||||
cancelled || delegate.isCancelled()
|
||||
}
|
||||
|
||||
def cancel(): Unit = {
|
||||
// the underlying Timeout will not become cancelled once the task has been started to run,
|
||||
// therefore we keep a flag here to make sure that rescheduling doesn't occur when cancelled
|
||||
cancelled = true
|
||||
// delegate is initially null, but this object will not be exposed to the world until after init
|
||||
delegate.cancel()
|
||||
}
|
||||
def isCancelled(): Boolean = get().isCancelled()
|
||||
def cancel(): Unit = getAndSet(ContinuousCancellable.cancelled).cancel()
|
||||
}
|
||||
|
||||
class DefaultCancellable(val timeout: HWTimeout) extends Cancellable {
|
||||
def cancel() {
|
||||
timeout.cancel()
|
||||
}
|
||||
|
||||
def isCancelled: Boolean = {
|
||||
timeout.isCancelled
|
||||
}
|
||||
private[akka] class DefaultCancellable(timeout: HWTimeout) extends AtomicReference[HWTimeout](timeout) with Cancellable {
|
||||
override def cancel(): Unit = getAndSet(ContinuousCancellable.cancelled).cancel()
|
||||
override def isCancelled: Boolean = get().isCancelled
|
||||
}
|
||||
|
|
|
|||
|
|
@ -56,7 +56,7 @@ trait Stash {
|
|||
|
||||
/* The capacity of the stash. Configured in the actor's dispatcher config.
|
||||
*/
|
||||
private val capacity = {
|
||||
private val capacity: Int = {
|
||||
val dispatcher = context.system.settings.config.getConfig(context.props.dispatcher)
|
||||
val config = dispatcher.withFallback(context.system.settings.config.getConfig("akka.actor.default-dispatcher"))
|
||||
config.getInt("stash-capacity")
|
||||
|
|
@ -125,4 +125,7 @@ An (unbounded) deque-based mailbox can be configured as follows:
|
|||
|
||||
}
|
||||
|
||||
/**
|
||||
* Is thrown when the size of the Stash exceeds the capacity of the Stash
|
||||
*/
|
||||
class StashOverflowException(message: String, cause: Throwable = null) extends AkkaException(message, cause)
|
||||
|
|
|
|||
|
|
@ -6,21 +6,28 @@ package akka.actor
|
|||
|
||||
import akka.japi.{ Creator, Option ⇒ JOption }
|
||||
import java.lang.reflect.{ InvocationTargetException, Method, InvocationHandler, Proxy }
|
||||
import akka.util.{ Timeout, NonFatal }
|
||||
import akka.util.{ Timeout, NonFatal, Duration }
|
||||
import java.util.concurrent.atomic.{ AtomicReference ⇒ AtomVar }
|
||||
import akka.dispatch._
|
||||
import java.util.concurrent.TimeoutException
|
||||
import java.util.concurrent.TimeUnit.MILLISECONDS
|
||||
import java.lang.IllegalStateException
|
||||
import akka.util.Duration
|
||||
import akka.actor.TypedActor.TypedActorInvocationHandler
|
||||
import akka.serialization.{ JavaSerializer, Serialization, SerializationExtension }
|
||||
import akka.serialization.{ JavaSerializer, SerializationExtension }
|
||||
import java.io.ObjectStreamException
|
||||
|
||||
/**
|
||||
* A TypedActorFactory is something that can created TypedActor instances.
|
||||
*/
|
||||
trait TypedActorFactory {
|
||||
|
||||
/**
|
||||
* Underlying dependency is to be able to create normal Actors
|
||||
*/
|
||||
protected def actorFactory: ActorRefFactory
|
||||
|
||||
/**
|
||||
* Underlying dependency to a TypedActorExtension, which can either be contextual or ActorSystem "global"
|
||||
*/
|
||||
protected def typedActor: TypedActorExtension
|
||||
|
||||
/**
|
||||
|
|
@ -80,6 +87,9 @@ trait TypedActorFactory {
|
|||
|
||||
}
|
||||
|
||||
/**
|
||||
* This represents the TypedActor Akka Extension, access to the functionality is done through a given ActorSystem.
|
||||
*/
|
||||
object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvider {
|
||||
override def get(system: ActorSystem): TypedActorExtension = super.get(system)
|
||||
|
||||
|
|
@ -145,8 +155,10 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
|
|||
|
||||
/**
|
||||
* Represents the serialized form of a MethodCall, uses readResolve and writeReplace to marshall the call
|
||||
*
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
case class SerializedMethodCall(ownerType: Class[_], methodName: String, parameterTypes: Array[Class[_]], serializedParameters: Array[(Int, Class[_], Array[Byte])]) {
|
||||
private[akka] case class SerializedMethodCall(ownerType: Class[_], methodName: String, parameterTypes: Array[Class[_]], serializedParameters: Array[(Int, Class[_], Array[Byte])]) {
|
||||
|
||||
//TODO implement writeObject and readObject to serialize
|
||||
//TODO Possible optimization is to special encode the parameter-types to conserve space
|
||||
|
|
@ -213,6 +225,8 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
|
|||
|
||||
/**
|
||||
* Implementation of TypedActor as an Actor
|
||||
*
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
private[akka] class TypedActor[R <: AnyRef, T <: R](val proxyVar: AtomVar[R], createInstance: ⇒ T) extends Actor {
|
||||
val me = try {
|
||||
|
|
@ -371,6 +385,9 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
|
|||
def postRestart(reason: Throwable): Unit
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
private[akka] class TypedActorInvocationHandler(@transient val extension: TypedActorExtension, @transient val actorVar: AtomVar[ActorRef], @transient val timeout: Timeout) extends InvocationHandler with Serializable {
|
||||
def actor = actorVar.get
|
||||
@throws(classOf[Throwable])
|
||||
|
|
@ -396,6 +413,9 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
|
|||
@throws(classOf[ObjectStreamException]) private def writeReplace(): AnyRef = SerializedTypedActorInvocationHandler(actor, timeout.duration)
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
private[akka] case class SerializedTypedActorInvocationHandler(val actor: ActorRef, val timeout: Duration) {
|
||||
@throws(classOf[ObjectStreamException]) private def readResolve(): AnyRef = JavaSerializer.currentSystem.value match {
|
||||
case null ⇒ throw new IllegalStateException("SerializedTypedActorInvocationHandler.readResolve requires that JavaSerializer.currentSystem.value is set to a non-null value")
|
||||
|
|
@ -569,12 +589,16 @@ case class TypedProps[T <: AnyRef] protected[TypedProps] (
|
|||
def withoutInterface(interface: Class[_ >: T]): TypedProps[T] =
|
||||
this.copy(interfaces = interfaces diff TypedProps.extractInterfaces(interface))
|
||||
|
||||
import akka.actor.{ Props ⇒ ActorProps }
|
||||
def actorProps(): ActorProps =
|
||||
if (dispatcher == ActorProps().dispatcher) ActorProps()
|
||||
else ActorProps(dispatcher = dispatcher)
|
||||
/**
|
||||
* Returns the akka.actor.Props representation of this TypedProps
|
||||
*/
|
||||
def actorProps(): Props = if (dispatcher == Props().dispatcher) Props() else Props(dispatcher = dispatcher)
|
||||
}
|
||||
|
||||
/**
|
||||
* ContextualTypedActorFactory allows TypedActors to create children, effectively forming the same Actor Supervision Hierarchies
|
||||
* as normal Actors can.
|
||||
*/
|
||||
case class ContextualTypedActorFactory(typedActor: TypedActorExtension, actorFactory: ActorContext) extends TypedActorFactory {
|
||||
override def getActorRefFor(proxy: AnyRef): ActorRef = typedActor.getActorRefFor(proxy)
|
||||
override def isTypedActor(proxyOrNot: AnyRef): Boolean = typedActor.isTypedActor(proxyOrNot)
|
||||
|
|
@ -607,7 +631,9 @@ class TypedActorExtension(system: ExtendedActorSystem) extends TypedActorFactory
|
|||
def isTypedActor(proxyOrNot: AnyRef): Boolean = invocationHandlerFor(proxyOrNot) ne null
|
||||
|
||||
// Private API
|
||||
|
||||
/**
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
private[akka] def createActorRefProxy[R <: AnyRef, T <: R](props: TypedProps[T], proxyVar: AtomVar[R], actorRef: ⇒ ActorRef): R = {
|
||||
//Warning, do not change order of the following statements, it's some elaborate chicken-n-egg handling
|
||||
val actorVar = new AtomVar[ActorRef](null)
|
||||
|
|
@ -631,6 +657,9 @@ class TypedActorExtension(system: ExtendedActorSystem) extends TypedActorFactory
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
private[akka] def invocationHandlerFor(typedActor_? : AnyRef): TypedActorInvocationHandler =
|
||||
if ((typedActor_? ne null) && Proxy.isProxyClass(typedActor_?.getClass)) typedActor_? match {
|
||||
case null ⇒ null
|
||||
|
|
|
|||
|
|
@ -93,11 +93,17 @@ import akka.japi.{ Creator }
|
|||
abstract class UntypedActor extends Actor {
|
||||
|
||||
/**
|
||||
* To be implemented by concrete UntypedActor. Defines the message handler.
|
||||
* To be implemented by concrete UntypedActor, this defines the behavior of the
|
||||
* UntypedActor.
|
||||
*/
|
||||
@throws(classOf[Exception])
|
||||
def onReceive(message: Any): Unit
|
||||
|
||||
/**
|
||||
* Returns this UntypedActor's UntypedActorContext
|
||||
* The UntypedActorContext is not thread safe so do not expose it outside of the
|
||||
* UntypedActor.
|
||||
*/
|
||||
def getContext(): UntypedActorContext = context.asInstanceOf[UntypedActorContext]
|
||||
|
||||
/**
|
||||
|
|
@ -150,9 +156,7 @@ abstract class UntypedActor extends Actor {
|
|||
*/
|
||||
override def postRestart(reason: Throwable): Unit = super.postRestart(reason)
|
||||
|
||||
final protected def receive = {
|
||||
case msg ⇒ onReceive(msg)
|
||||
}
|
||||
final def receive = { case msg ⇒ onReceive(msg) }
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -7,24 +7,4 @@ package akka
|
|||
package object actor {
|
||||
implicit def actorRef2Scala(ref: ActorRef): ScalaActorRef = ref.asInstanceOf[ScalaActorRef]
|
||||
implicit def scala2ActorRef(ref: ScalaActorRef): ActorRef = ref.asInstanceOf[ActorRef]
|
||||
|
||||
type Uuid = com.eaio.uuid.UUID
|
||||
|
||||
def newUuid(): Uuid = new Uuid()
|
||||
|
||||
def uuidFrom(time: Long, clockSeqAndNode: Long): Uuid = new Uuid(time, clockSeqAndNode)
|
||||
|
||||
def uuidFrom(uuid: String): Uuid = new Uuid(uuid)
|
||||
|
||||
def simpleName(obj: AnyRef): String = {
|
||||
val n = obj.getClass.getName
|
||||
val i = n.lastIndexOf('.')
|
||||
n.substring(i + 1)
|
||||
}
|
||||
|
||||
def simpleName(clazz: Class[_]): String = {
|
||||
val n = clazz.getName
|
||||
val i = n.lastIndexOf('.')
|
||||
n.substring(i + 1)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,15 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.config
|
||||
|
||||
import akka.AkkaException
|
||||
|
||||
class ConfigurationException(message: String, cause: Throwable = null) extends AkkaException(message, cause) {
|
||||
def this(msg: String) = this(msg, null);
|
||||
}
|
||||
|
||||
class ModuleNotAvailableException(message: String, cause: Throwable = null) extends AkkaException(message, cause) {
|
||||
def this(msg: String) = this(msg, null);
|
||||
}
|
||||
|
|
@ -33,7 +33,10 @@ final case class Envelope(val message: Any, val sender: ActorRef)(system: ActorS
|
|||
}
|
||||
}
|
||||
|
||||
object SystemMessage {
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object SystemMessage {
|
||||
@tailrec
|
||||
final def size(list: SystemMessage, acc: Int = 0): Int = {
|
||||
if (list eq null) acc else size(list.next, acc + 1)
|
||||
|
|
@ -59,33 +62,57 @@ object SystemMessage {
|
|||
* system messages is handled in a single thread only and not ever passed around,
|
||||
* hence no further synchronization is needed.
|
||||
*
|
||||
* INTERNAL API
|
||||
*
|
||||
* ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
|
||||
*/
|
||||
sealed trait SystemMessage extends PossiblyHarmful {
|
||||
private[akka] sealed trait SystemMessage extends PossiblyHarmful {
|
||||
@transient
|
||||
var next: SystemMessage = _
|
||||
}
|
||||
case class Create() extends SystemMessage // send to self from Dispatcher.register
|
||||
case class Recreate(cause: Throwable) extends SystemMessage // sent to self from ActorCell.restart
|
||||
case class Suspend() extends SystemMessage // sent to self from ActorCell.suspend
|
||||
case class Resume() extends SystemMessage // sent to self from ActorCell.resume
|
||||
case class Terminate() extends SystemMessage // sent to self from ActorCell.stop
|
||||
case class Supervise(child: ActorRef) extends SystemMessage // sent to supervisor ActorRef from ActorCell.start
|
||||
case class ChildTerminated(child: ActorRef) extends SystemMessage // sent to supervisor from ActorCell.doTerminate
|
||||
case class Link(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.watch
|
||||
case class Unlink(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.unwatch
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] case class Create() extends SystemMessage // send to self from Dispatcher.register
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] case class Recreate(cause: Throwable) extends SystemMessage // sent to self from ActorCell.restart
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] case class Suspend() extends SystemMessage // sent to self from ActorCell.suspend
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] case class Resume() extends SystemMessage // sent to self from ActorCell.resume
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] case class Terminate() extends SystemMessage // sent to self from ActorCell.stop
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] case class Supervise(child: ActorRef) extends SystemMessage // sent to supervisor ActorRef from ActorCell.start
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] case class ChildTerminated(child: ActorRef) extends SystemMessage // sent to supervisor from ActorCell.doTerminate
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] case class Link(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.watch
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] case class Unlink(subject: ActorRef) extends SystemMessage // sent to self from ActorCell.unwatch
|
||||
|
||||
final case class TaskInvocation(eventStream: EventStream, runnable: Runnable, cleanup: () ⇒ Unit) extends Runnable {
|
||||
def run() {
|
||||
try {
|
||||
runnable.run()
|
||||
} catch {
|
||||
case NonFatal(e) ⇒
|
||||
eventStream.publish(Error(e, "TaskInvocation", this.getClass, e.getMessage))
|
||||
} finally {
|
||||
cleanup()
|
||||
}
|
||||
}
|
||||
def run(): Unit =
|
||||
try runnable.run() catch {
|
||||
case NonFatal(e) ⇒ eventStream.publish(Error(e, "TaskInvocation", this.getClass, e.getMessage))
|
||||
} finally cleanup()
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -170,10 +197,16 @@ trait ExecutionContext {
|
|||
def reportFailure(t: Throwable): Unit
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] trait LoadMetrics { self: Executor ⇒
|
||||
def atFullThrottle(): Boolean
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object MessageDispatcher {
|
||||
val UNSCHEDULED = 0 //WARNING DO NOT CHANGE THE VALUE OF THIS: It relies on the faster init of 0 in AbstractMessageDispatcher
|
||||
val SCHEDULED = 1
|
||||
|
|
@ -181,7 +214,7 @@ private[akka] object MessageDispatcher {
|
|||
|
||||
// dispatcher debugging helper using println (see below)
|
||||
// since this is a compile-time constant, scalac will elide code behind if (MessageDispatcher.debug) (RK checked with 2.9.1)
|
||||
final val debug = false
|
||||
final val debug = false // Deliberately without type ascription to make it a compile-time constant
|
||||
lazy val actors = new Index[MessageDispatcher, ActorRef](16, _ compareTo _)
|
||||
def printActors: Unit = if (debug) {
|
||||
for {
|
||||
|
|
@ -228,7 +261,7 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
|
|||
/**
|
||||
* Creates and returns a mailbox for the given actor.
|
||||
*/
|
||||
protected[akka] def createMailbox(actor: ActorCell): Mailbox
|
||||
protected[akka] def createMailbox(actor: ActorCell): Mailbox //FIXME should this really be private[akka]?
|
||||
|
||||
/**
|
||||
* Identifier of this dispatcher, corresponds to the full key
|
||||
|
|
@ -255,7 +288,7 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
|
|||
ifSensibleToDoSoThenScheduleShutdown()
|
||||
}
|
||||
|
||||
final def execute(runnable: Runnable) {
|
||||
final def execute(runnable: Runnable): Unit = {
|
||||
val invocation = TaskInvocation(eventStream, runnable, taskCleanup)
|
||||
addInhabitants(+1)
|
||||
try {
|
||||
|
|
@ -300,6 +333,8 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
|
|||
|
||||
/**
|
||||
* If you override it, you must call it. But only ever once. See "attach" for only invocation.
|
||||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
protected[akka] def register(actor: ActorCell) {
|
||||
if (debug) actors.put(this, actor.self)
|
||||
|
|
@ -308,6 +343,8 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
|
|||
|
||||
/**
|
||||
* If you override it, you must call it. But only ever once. See "detach" for the only invocation
|
||||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
protected[akka] def unregister(actor: ActorCell) {
|
||||
if (debug) actors.remove(this, actor.self)
|
||||
|
|
@ -340,6 +377,8 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
|
|||
* When the dispatcher no longer has any actors registered, how long will it wait until it shuts itself down,
|
||||
* defaulting to your akka configs "akka.actor.default-dispatcher.shutdown-timeout" or default specified in
|
||||
* reference.conf
|
||||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
protected[akka] def shutdownTimeout: Duration
|
||||
|
||||
|
|
@ -362,36 +401,59 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
|
|||
}
|
||||
|
||||
/**
|
||||
* Will be called when the dispatcher is to queue an invocation for execution
|
||||
* Will be called when the dispatcher is to queue an invocation for execution
|
||||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
protected[akka] def systemDispatch(receiver: ActorCell, invocation: SystemMessage)
|
||||
|
||||
/**
|
||||
* Will be called when the dispatcher is to queue an invocation for execution
|
||||
* Will be called when the dispatcher is to queue an invocation for execution
|
||||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope)
|
||||
|
||||
/**
|
||||
* Suggest to register the provided mailbox for execution
|
||||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
protected[akka] def registerForExecution(mbox: Mailbox, hasMessageHint: Boolean, hasSystemMessageHint: Boolean): Boolean
|
||||
|
||||
// TODO check whether this should not actually be a property of the mailbox
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
protected[akka] def throughput: Int
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
protected[akka] def throughputDeadlineTime: Duration
|
||||
|
||||
@inline
|
||||
protected[akka] final val isThroughputDeadlineTimeDefined = throughputDeadlineTime.toMillis > 0
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@inline protected[akka] final val isThroughputDeadlineTimeDefined = throughputDeadlineTime.toMillis > 0
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
protected[akka] def executeTask(invocation: TaskInvocation)
|
||||
|
||||
/**
|
||||
* Called one time every time an actor is detached from this dispatcher and this dispatcher has no actors left attached
|
||||
* Must be idempotent
|
||||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
protected[akka] def shutdown(): Unit
|
||||
}
|
||||
|
||||
/**
|
||||
* An ExecutorServiceConfigurator is a class that given some prerequisites and a configuration can create instances of ExecutorService
|
||||
*/
|
||||
abstract class ExecutorServiceConfigurator(config: Config, prerequisites: DispatcherPrerequisites) extends ExecutorServiceFactoryProvider
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -35,12 +35,35 @@ class BalancingDispatcher(
|
|||
attemptTeamWork: Boolean)
|
||||
extends Dispatcher(_prerequisites, _id, throughput, throughputDeadlineTime, mailboxType, _executorServiceFactoryProvider, _shutdownTimeout) {
|
||||
|
||||
val team = new ConcurrentSkipListSet[ActorCell](
|
||||
/**
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
private[akka] val team = new ConcurrentSkipListSet[ActorCell](
|
||||
Helpers.identityHashComparator(new Comparator[ActorCell] {
|
||||
def compare(l: ActorCell, r: ActorCell) = l.self.path compareTo r.self.path
|
||||
}))
|
||||
|
||||
val messageQueue: MessageQueue = mailboxType.create(None)
|
||||
/**
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
private[akka] val messageQueue: MessageQueue = mailboxType.create(None)
|
||||
|
||||
private class SharingMailbox(_actor: ActorCell, _messageQueue: MessageQueue) extends Mailbox(_actor, _messageQueue) with DefaultSystemMessageQueue {
|
||||
override def cleanUp(): Unit = {
|
||||
//Don't call the original implementation of this since it scraps all messages, and we don't want to do that
|
||||
if (hasSystemMessages) {
|
||||
val dlq = actor.systemImpl.deadLetterMailbox
|
||||
var message = systemDrain()
|
||||
while (message ne null) {
|
||||
// message must be “virgin” before being able to systemEnqueue again
|
||||
val next = message.next
|
||||
message.next = null
|
||||
dlq.systemEnqueue(actor.self, message)
|
||||
message = next
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected[akka] override def createMailbox(actor: ActorCell): Mailbox = new SharingMailbox(actor, messageQueue)
|
||||
|
||||
|
|
@ -64,7 +87,7 @@ class BalancingDispatcher(
|
|||
@tailrec def scheduleOne(i: Iterator[ActorCell] = team.iterator): Unit =
|
||||
if (messageQueue.hasMessages
|
||||
&& i.hasNext
|
||||
&& (executorService.get().executor match {
|
||||
&& (executorService.executor match {
|
||||
case lm: LoadMetrics ⇒ lm.atFullThrottle == false
|
||||
case other ⇒ true
|
||||
})
|
||||
|
|
@ -74,22 +97,3 @@ class BalancingDispatcher(
|
|||
scheduleOne()
|
||||
}
|
||||
}
|
||||
|
||||
class SharingMailbox(_actor: ActorCell, _messageQueue: MessageQueue)
|
||||
extends Mailbox(_actor, _messageQueue) with DefaultSystemMessageQueue {
|
||||
|
||||
override def cleanUp(): Unit = {
|
||||
//Don't call the original implementation of this since it scraps all messages, and we don't want to do that
|
||||
if (hasSystemMessages) {
|
||||
val dlq = actor.systemImpl.deadLetterMailbox
|
||||
var message = systemDrain()
|
||||
while (message ne null) {
|
||||
// message must be “virgin” before being able to systemEnqueue again
|
||||
val next = message.next
|
||||
message.next = null
|
||||
dlq.systemEnqueue(actor.self, message)
|
||||
message = next
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -9,6 +9,7 @@ import java.util.concurrent.atomic.AtomicReference
|
|||
import akka.actor.ActorCell
|
||||
import akka.util.Duration
|
||||
import java.util.concurrent._
|
||||
import akka.event.Logging
|
||||
|
||||
/**
|
||||
* The event-based ``Dispatcher`` binds a set of Actors to a thread pool backed up by a
|
||||
|
|
@ -32,31 +33,44 @@ class Dispatcher(
|
|||
val shutdownTimeout: Duration)
|
||||
extends MessageDispatcher(_prerequisites) {
|
||||
|
||||
protected val executorServiceFactory: ExecutorServiceFactory =
|
||||
executorServiceFactoryProvider.createExecutorServiceFactory(id, prerequisites.threadFactory)
|
||||
private class LazyExecutorServiceDelegate(factory: ExecutorServiceFactory) extends ExecutorServiceDelegate {
|
||||
lazy val executor: ExecutorService = factory.createExecutorService
|
||||
def copy(): LazyExecutorServiceDelegate = new LazyExecutorServiceDelegate(factory)
|
||||
}
|
||||
|
||||
protected val executorService = new AtomicReference[ExecutorServiceDelegate](
|
||||
new ExecutorServiceDelegate { lazy val executor = executorServiceFactory.createExecutorService })
|
||||
@volatile private var executorServiceDelegate: LazyExecutorServiceDelegate =
|
||||
new LazyExecutorServiceDelegate(executorServiceFactoryProvider.createExecutorServiceFactory(id, prerequisites.threadFactory))
|
||||
|
||||
protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = {
|
||||
protected final def executorService: ExecutorServiceDelegate = executorServiceDelegate
|
||||
|
||||
/**
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope): Unit = {
|
||||
val mbox = receiver.mailbox
|
||||
mbox.enqueue(receiver.self, invocation)
|
||||
registerForExecution(mbox, true, false)
|
||||
}
|
||||
|
||||
protected[akka] def systemDispatch(receiver: ActorCell, invocation: SystemMessage) = {
|
||||
/**
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
protected[akka] def systemDispatch(receiver: ActorCell, invocation: SystemMessage): Unit = {
|
||||
val mbox = receiver.mailbox
|
||||
mbox.systemEnqueue(receiver.self, invocation)
|
||||
registerForExecution(mbox, false, true)
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
protected[akka] def executeTask(invocation: TaskInvocation) {
|
||||
try {
|
||||
executorService.get() execute invocation
|
||||
executorService execute invocation
|
||||
} catch {
|
||||
case e: RejectedExecutionException ⇒
|
||||
try {
|
||||
executorService.get() execute invocation
|
||||
executorService execute invocation
|
||||
} catch {
|
||||
case e2: RejectedExecutionException ⇒
|
||||
prerequisites.eventStream.publish(Error(e, getClass.getName, getClass, "executeTask was rejected twice!"))
|
||||
|
|
@ -65,26 +79,39 @@ class Dispatcher(
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
protected[akka] def createMailbox(actor: ActorCell): Mailbox = new Mailbox(actor, mailboxType.create(Some(actor))) with DefaultSystemMessageQueue
|
||||
|
||||
protected[akka] def shutdown: Unit =
|
||||
Option(executorService.getAndSet(new ExecutorServiceDelegate {
|
||||
lazy val executor = executorServiceFactory.createExecutorService
|
||||
})) foreach { _.shutdown() }
|
||||
/**
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
protected[akka] def shutdown: Unit = {
|
||||
val newDelegate = executorServiceDelegate.copy() // Doesn't matter which one we copy
|
||||
val es = synchronized { // FIXME getAndSet using ARFU or Unsafe
|
||||
val service = executorServiceDelegate
|
||||
executorServiceDelegate = newDelegate // just a quick getAndSet
|
||||
service
|
||||
}
|
||||
es.shutdown()
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns if it was registered
|
||||
*
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
protected[akka] override def registerForExecution(mbox: Mailbox, hasMessageHint: Boolean, hasSystemMessageHint: Boolean): Boolean = {
|
||||
if (mbox.canBeScheduledForExecution(hasMessageHint, hasSystemMessageHint)) { //This needs to be here to ensure thread safety and no races
|
||||
if (mbox.setAsScheduled()) {
|
||||
try {
|
||||
executorService.get() execute mbox
|
||||
executorService execute mbox
|
||||
true
|
||||
} catch {
|
||||
case e: RejectedExecutionException ⇒
|
||||
try {
|
||||
executorService.get() execute mbox
|
||||
executorService execute mbox
|
||||
true
|
||||
} catch { //Retry once
|
||||
case e: RejectedExecutionException ⇒
|
||||
|
|
@ -97,7 +124,7 @@ class Dispatcher(
|
|||
} else false
|
||||
}
|
||||
|
||||
override val toString = getClass.getSimpleName + "[" + id + "]"
|
||||
override val toString: String = Logging.simpleName(this) + "[" + id + "]"
|
||||
}
|
||||
|
||||
object PriorityGenerator {
|
||||
|
|
|
|||
|
|
@ -5,17 +5,15 @@
|
|||
package akka.dispatch
|
||||
|
||||
import java.util.concurrent.{ ConcurrentHashMap, TimeUnit, ThreadFactory }
|
||||
|
||||
import scala.collection.JavaConverters.mapAsJavaMapConverter
|
||||
|
||||
import com.typesafe.config.{ ConfigFactory, Config }
|
||||
|
||||
import Dispatchers.DefaultDispatcherId
|
||||
import akka.actor.{ Scheduler, DynamicAccess, ActorSystem }
|
||||
import akka.event.Logging.Warning
|
||||
import akka.event.EventStream
|
||||
import akka.util.Duration
|
||||
|
||||
/**
|
||||
* DispatcherPrerequisites represents useful contextual pieces when constructing a MessageDispatcher
|
||||
*/
|
||||
trait DispatcherPrerequisites {
|
||||
def threadFactory: ThreadFactory
|
||||
def eventStream: EventStream
|
||||
|
|
@ -25,7 +23,10 @@ trait DispatcherPrerequisites {
|
|||
def settings: ActorSystem.Settings
|
||||
}
|
||||
|
||||
case class DefaultDispatcherPrerequisites(
|
||||
/**
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
private[akka] case class DefaultDispatcherPrerequisites(
|
||||
val threadFactory: ThreadFactory,
|
||||
val eventStream: EventStream,
|
||||
val deadLetterMailbox: Mailbox,
|
||||
|
|
@ -96,6 +97,7 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc
|
|||
}
|
||||
}
|
||||
|
||||
//INTERNAL API
|
||||
private def config(id: String): Config = {
|
||||
import scala.collection.JavaConverters._
|
||||
def simpleName = id.substring(id.lastIndexOf('.') + 1)
|
||||
|
|
@ -105,12 +107,13 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc
|
|||
.withFallback(defaultDispatcherConfig)
|
||||
}
|
||||
|
||||
//INTERNAL API
|
||||
private def idConfig(id: String): Config = {
|
||||
import scala.collection.JavaConverters._
|
||||
ConfigFactory.parseMap(Map("id" -> id).asJava)
|
||||
}
|
||||
|
||||
/*
|
||||
/**
|
||||
* Creates a dispatcher from a Config. Internal test purpose only.
|
||||
*
|
||||
* ex: from(config.getConfig(id))
|
||||
|
|
@ -119,18 +122,22 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc
|
|||
*
|
||||
* Throws: IllegalArgumentException if the value of "type" is not valid
|
||||
* IllegalArgumentException if it cannot create the MessageDispatcherConfigurator
|
||||
*
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
private[akka] def from(cfg: Config): MessageDispatcher = {
|
||||
configuratorFrom(cfg).dispatcher()
|
||||
}
|
||||
private[akka] def from(cfg: Config): MessageDispatcher = configuratorFrom(cfg).dispatcher()
|
||||
|
||||
/*
|
||||
private[akka] def isBalancingDispatcher(id: String): Boolean = settings.config.hasPath(id) && config(id).getString("type") == "BalancingDispatcher"
|
||||
|
||||
/**
|
||||
* Creates a MessageDispatcherConfigurator from a Config.
|
||||
*
|
||||
* The Config must also contain a `id` property, which is the identifier of the dispatcher.
|
||||
*
|
||||
* Throws: IllegalArgumentException if the value of "type" is not valid
|
||||
* IllegalArgumentException if it cannot create the MessageDispatcherConfigurator
|
||||
*
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
private def configuratorFrom(cfg: Config): MessageDispatcherConfigurator = {
|
||||
if (!cfg.hasPath("id")) throw new IllegalArgumentException("Missing dispatcher 'id' property in config: " + cfg.root.render)
|
||||
|
|
@ -208,7 +215,7 @@ class BalancingDispatcherConfigurator(config: Config, prerequisites: DispatcherP
|
|||
class PinnedDispatcherConfigurator(config: Config, prerequisites: DispatcherPrerequisites)
|
||||
extends MessageDispatcherConfigurator(config, prerequisites) {
|
||||
|
||||
val threadPoolConfig: ThreadPoolConfig = configureExecutor() match {
|
||||
private val threadPoolConfig: ThreadPoolConfig = configureExecutor() match {
|
||||
case e: ThreadPoolExecutorConfigurator ⇒ e.threadPoolConfig
|
||||
case other ⇒
|
||||
prerequisites.eventStream.publish(
|
||||
|
|
|
|||
|
|
@ -14,9 +14,15 @@ import akka.actor.ActorContext
|
|||
import com.typesafe.config.Config
|
||||
import akka.actor.ActorSystem
|
||||
|
||||
/**
|
||||
* This exception normally is thrown when a bounded mailbox is over capacity
|
||||
*/
|
||||
class MessageQueueAppendFailedException(message: String, cause: Throwable = null) extends AkkaException(message, cause)
|
||||
|
||||
object Mailbox {
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object Mailbox {
|
||||
|
||||
type Status = Int
|
||||
|
||||
|
|
@ -25,21 +31,22 @@ object Mailbox {
|
|||
*/
|
||||
|
||||
// primary status: only first three
|
||||
final val Open = 0 // _status is not initialized in AbstractMailbox, so default must be zero!
|
||||
final val Suspended = 1
|
||||
final val Closed = 2
|
||||
final val Open = 0 // _status is not initialized in AbstractMailbox, so default must be zero! Deliberately without type ascription to make it a compile-time constant
|
||||
final val Suspended = 1 // Deliberately without type ascription to make it a compile-time constant
|
||||
final val Closed = 2 // Deliberately without type ascription to make it a compile-time constant
|
||||
// secondary status: Scheduled bit may be added to Open/Suspended
|
||||
final val Scheduled = 4
|
||||
final val Scheduled = 4 // Deliberately without type ascription to make it a compile-time constant
|
||||
|
||||
// mailbox debugging helper using println (see below)
|
||||
// since this is a compile-time constant, scalac will elide code behind if (Mailbox.debug) (RK checked with 2.9.1)
|
||||
final val debug = false
|
||||
final val debug = false // Deliberately without type ascription to make it a compile-time constant
|
||||
}
|
||||
|
||||
/**
|
||||
* Mailbox and InternalMailbox is separated in two classes because ActorCell is needed for implementation,
|
||||
* but can't be exposed to user defined mailbox subclasses.
|
||||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: MessageQueue)
|
||||
extends SystemMessageQueue with Runnable {
|
||||
|
|
@ -244,6 +251,11 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A MessageQueue is one of the core components in forming an Akka Mailbox.
|
||||
* The MessageQueue is where the normal messages that are sent to Actors will be enqueued (and subsequently dequeued)
|
||||
* It needs to atleast support N producers and 1 consumer thread-safely.
|
||||
*/
|
||||
trait MessageQueue {
|
||||
/**
|
||||
* Try to enqueue the message to this queue, or throw an exception.
|
||||
|
|
@ -277,7 +289,7 @@ trait MessageQueue {
|
|||
}
|
||||
|
||||
/**
|
||||
* Internal mailbox implementation detail.
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
private[akka] trait SystemMessageQueue {
|
||||
/**
|
||||
|
|
@ -294,7 +306,7 @@ private[akka] trait SystemMessageQueue {
|
|||
}
|
||||
|
||||
/**
|
||||
* Internal mailbox implementation detail.
|
||||
* INTERNAL USE ONLY
|
||||
*/
|
||||
private[akka] trait DefaultSystemMessageQueue { self: Mailbox ⇒
|
||||
|
||||
|
|
@ -325,6 +337,9 @@ private[akka] trait DefaultSystemMessageQueue { self: Mailbox ⇒
|
|||
def hasSystemMessages: Boolean = systemQueueGet ne null
|
||||
}
|
||||
|
||||
/**
|
||||
* A QueueBasedMessageQueue is a MessageQueue backed by a java.util.Queue
|
||||
*/
|
||||
trait QueueBasedMessageQueue extends MessageQueue {
|
||||
def queue: Queue[Envelope]
|
||||
def numberOfMessages = queue.size
|
||||
|
|
@ -340,11 +355,19 @@ trait QueueBasedMessageQueue extends MessageQueue {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* UnboundedMessageQueueSemantics adds unbounded semantics to a QueueBasedMessageQueue,
|
||||
* i.e. a non-blocking enqueue and dequeue.
|
||||
*/
|
||||
trait UnboundedMessageQueueSemantics extends QueueBasedMessageQueue {
|
||||
def enqueue(receiver: ActorRef, handle: Envelope): Unit = queue add handle
|
||||
def dequeue(): Envelope = queue.poll()
|
||||
}
|
||||
|
||||
/**
|
||||
* BoundedMessageQueueSemantics adds bounded semantics to a QueueBasedMessageQueue,
|
||||
* i.e. blocking enqueue with timeout
|
||||
*/
|
||||
trait BoundedMessageQueueSemantics extends QueueBasedMessageQueue {
|
||||
def pushTimeOut: Duration
|
||||
override def queue: BlockingQueue[Envelope]
|
||||
|
|
@ -360,17 +383,28 @@ trait BoundedMessageQueueSemantics extends QueueBasedMessageQueue {
|
|||
def dequeue(): Envelope = queue.poll()
|
||||
}
|
||||
|
||||
/**
|
||||
* DequeBasedMessageQueue refines QueueBasedMessageQueue to be backed by a java.util.Deque
|
||||
*/
|
||||
trait DequeBasedMessageQueue extends QueueBasedMessageQueue {
|
||||
def queue: Deque[Envelope]
|
||||
def enqueueFirst(receiver: ActorRef, handle: Envelope): Unit
|
||||
}
|
||||
|
||||
/**
|
||||
* UnboundedDequeBasedMessageQueueSemantics adds unbounded semantics to a DequeBasedMessageQueue,
|
||||
* i.e. a non-blocking enqueue and dequeue.
|
||||
*/
|
||||
trait UnboundedDequeBasedMessageQueueSemantics extends DequeBasedMessageQueue {
|
||||
def enqueue(receiver: ActorRef, handle: Envelope): Unit = queue add handle
|
||||
def enqueueFirst(receiver: ActorRef, handle: Envelope): Unit = queue addFirst handle
|
||||
def dequeue(): Envelope = queue.poll()
|
||||
}
|
||||
|
||||
/**
|
||||
* BoundedMessageQueueSemantics adds bounded semantics to a DequeBasedMessageQueue,
|
||||
* i.e. blocking enqueue with timeout
|
||||
*/
|
||||
trait BoundedDequeBasedMessageQueueSemantics extends DequeBasedMessageQueue {
|
||||
def pushTimeOut: Duration
|
||||
override def queue: BlockingDeque[Envelope]
|
||||
|
|
@ -393,14 +427,14 @@ trait BoundedDequeBasedMessageQueueSemantics extends DequeBasedMessageQueue {
|
|||
}
|
||||
|
||||
/**
|
||||
* Mailbox configuration.
|
||||
* MailboxType is a factory to create MessageQueues for an optionally provided ActorContext
|
||||
*/
|
||||
trait MailboxType {
|
||||
def create(owner: Option[ActorContext]): MessageQueue
|
||||
}
|
||||
|
||||
/**
|
||||
* It's a case class for Java (new UnboundedMailbox)
|
||||
* UnboundedMailbox is the default unbounded MailboxType used by Akka Actors.
|
||||
*/
|
||||
case class UnboundedMailbox() extends MailboxType {
|
||||
|
||||
|
|
@ -412,6 +446,9 @@ case class UnboundedMailbox() extends MailboxType {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* BoundedMailbox is the default bounded MailboxType used by Akka Actors.
|
||||
*/
|
||||
case class BoundedMailbox( final val capacity: Int, final val pushTimeOut: Duration) extends MailboxType {
|
||||
|
||||
def this(settings: ActorSystem.Settings, config: Config) = this(config.getInt("mailbox-capacity"),
|
||||
|
|
@ -428,17 +465,20 @@ case class BoundedMailbox( final val capacity: Int, final val pushTimeOut: Durat
|
|||
}
|
||||
|
||||
/**
|
||||
* Extend me to provide the comparator
|
||||
* UnboundedPriorityMailbox is an unbounded mailbox that allows for priorization of its contents.
|
||||
* Extend this class and provide the Comparator in the constructor.
|
||||
*/
|
||||
class UnboundedPriorityMailbox( final val cmp: Comparator[Envelope]) extends MailboxType {
|
||||
class UnboundedPriorityMailbox( final val cmp: Comparator[Envelope], final val initialCapacity: Int) extends MailboxType {
|
||||
def this(cmp: Comparator[Envelope]) = this(cmp, 11)
|
||||
final override def create(owner: Option[ActorContext]): MessageQueue =
|
||||
new PriorityBlockingQueue[Envelope](11, cmp) with QueueBasedMessageQueue with UnboundedMessageQueueSemantics {
|
||||
new PriorityBlockingQueue[Envelope](initialCapacity, cmp) with QueueBasedMessageQueue with UnboundedMessageQueueSemantics {
|
||||
final def queue: Queue[Envelope] = this
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Extend me to provide the comparator
|
||||
* BoundedPriorityMailbox is a bounded mailbox that allows for priorization of its contents.
|
||||
* Extend this class and provide the Comparator in the constructor.
|
||||
*/
|
||||
class BoundedPriorityMailbox( final val cmp: Comparator[Envelope], final val capacity: Int, final val pushTimeOut: Duration) extends MailboxType {
|
||||
|
||||
|
|
@ -452,6 +492,9 @@ class BoundedPriorityMailbox( final val cmp: Comparator[Envelope], final val cap
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* UnboundedDequeBasedMailbox is an unbounded MailboxType, backed by a Deque.
|
||||
*/
|
||||
case class UnboundedDequeBasedMailbox() extends MailboxType {
|
||||
|
||||
def this(settings: ActorSystem.Settings, config: Config) = this()
|
||||
|
|
@ -462,6 +505,9 @@ case class UnboundedDequeBasedMailbox() extends MailboxType {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* BoundedDequeBasedMailbox is an bounded MailboxType, backed by a Deque.
|
||||
*/
|
||||
case class BoundedDequeBasedMailbox( final val capacity: Int, final val pushTimeOut: Duration) extends MailboxType {
|
||||
|
||||
def this(settings: ActorSystem.Settings, config: Config) = this(config.getInt("mailbox-capacity"),
|
||||
|
|
|
|||
|
|
@ -12,7 +12,7 @@ import akka.actor._
|
|||
* A failed subscribe should also only mean that the Classifier (ActorRef) that is listened to is already shut down
|
||||
* See LocalDeathWatch for semantics
|
||||
*/
|
||||
trait DeathWatch extends ActorEventBus with ActorClassifier {
|
||||
abstract class DeathWatch extends ActorEventBus with ActorClassifier {
|
||||
type Event = Terminated
|
||||
|
||||
protected final def classify(event: Event): Classifier = event.actor
|
||||
|
|
|
|||
|
|
@ -182,10 +182,9 @@ trait SubchannelClassification { this: EventBus ⇒
|
|||
*/
|
||||
trait ScanningClassification { self: EventBus ⇒
|
||||
protected final val subscribers = new ConcurrentSkipListSet[(Classifier, Subscriber)](new Comparator[(Classifier, Subscriber)] {
|
||||
def compare(a: (Classifier, Subscriber), b: (Classifier, Subscriber)): Int = {
|
||||
val cM = compareClassifiers(a._1, b._1)
|
||||
if (cM != 0) cM
|
||||
else compareSubscribers(a._2, b._2)
|
||||
def compare(a: (Classifier, Subscriber), b: (Classifier, Subscriber)): Int = compareClassifiers(a._1, b._1) match {
|
||||
case 0 ⇒ compareSubscribers(a._2, b._2)
|
||||
case other ⇒ other
|
||||
}
|
||||
})
|
||||
|
||||
|
|
@ -238,7 +237,7 @@ trait ActorClassification { this: ActorEventBus with ActorClassifier ⇒
|
|||
import java.util.concurrent.ConcurrentHashMap
|
||||
import scala.annotation.tailrec
|
||||
private val empty = TreeSet.empty[ActorRef]
|
||||
protected val mappings = new ConcurrentHashMap[ActorRef, TreeSet[ActorRef]](mapSize)
|
||||
private val mappings = new ConcurrentHashMap[ActorRef, TreeSet[ActorRef]](mapSize)
|
||||
|
||||
@tailrec
|
||||
protected final def associate(monitored: ActorRef, monitor: ActorRef): Boolean = {
|
||||
|
|
@ -320,9 +319,9 @@ trait ActorClassification { this: ActorEventBus with ActorClassifier ⇒
|
|||
*/
|
||||
protected def mapSize: Int
|
||||
|
||||
def publish(event: Event): Unit = {
|
||||
val receivers = mappings.get(classify(event))
|
||||
if (receivers ne null) receivers foreach { _ ! event }
|
||||
def publish(event: Event): Unit = mappings.get(classify(event)) match {
|
||||
case null ⇒ ()
|
||||
case some ⇒ some foreach { _ ! event }
|
||||
}
|
||||
|
||||
def subscribe(subscriber: Subscriber, to: Classifier): Boolean = associate(to, subscriber)
|
||||
|
|
|
|||
|
|
@ -3,7 +3,8 @@
|
|||
*/
|
||||
package akka.event
|
||||
|
||||
import akka.actor.{ ActorRef, ActorSystem, simpleName }
|
||||
import akka.actor.{ ActorRef, ActorSystem }
|
||||
import akka.event.Logging.simpleName
|
||||
import akka.util.Subclassification
|
||||
|
||||
object EventStream {
|
||||
|
|
|
|||
|
|
@ -4,12 +4,10 @@
|
|||
package akka.event
|
||||
|
||||
import akka.actor._
|
||||
import akka.AkkaException
|
||||
import akka.{ ConfigurationException, AkkaException }
|
||||
import akka.actor.ActorSystem.Settings
|
||||
import akka.config.ConfigurationException
|
||||
import akka.util.ReentrantGuard
|
||||
import akka.util.{ Timeout, ReentrantGuard }
|
||||
import akka.util.duration._
|
||||
import akka.util.Timeout
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import scala.util.control.NoStackTrace
|
||||
import java.util.concurrent.TimeoutException
|
||||
|
|
@ -31,7 +29,7 @@ trait LoggingBus extends ActorEventBus {
|
|||
|
||||
import Logging._
|
||||
|
||||
private val guard = new ReentrantGuard
|
||||
private val guard = new ReentrantGuard //Switch to ReentrantReadWrite
|
||||
private var loggers = Seq.empty[ActorRef]
|
||||
private var _logLevel: LogLevel = _
|
||||
|
||||
|
|
@ -99,7 +97,7 @@ trait LoggingBus extends ActorEventBus {
|
|||
val myloggers =
|
||||
for {
|
||||
loggerName ← defaultLoggers
|
||||
if loggerName != StandardOutLoggerName
|
||||
if loggerName != StandardOutLogger.getClass.getName
|
||||
} yield {
|
||||
try {
|
||||
system.dynamicAccess.getClassFor[Actor](loggerName) match {
|
||||
|
|
@ -131,7 +129,7 @@ trait LoggingBus extends ActorEventBus {
|
|||
case _: InvalidActorNameException ⇒ // ignore if it is already running
|
||||
}
|
||||
publish(Debug(logName, this.getClass, "Default Loggers started"))
|
||||
if (!(defaultLoggers contains StandardOutLoggerName)) {
|
||||
if (!(defaultLoggers contains StandardOutLogger.getClass.getName)) {
|
||||
unsubscribe(StandardOutLogger)
|
||||
}
|
||||
} catch {
|
||||
|
|
@ -165,6 +163,9 @@ trait LoggingBus extends ActorEventBus {
|
|||
publish(Debug(simpleName(this), this.getClass, "all default loggers stopped"))
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private def addLogger(system: ActorSystemImpl, clazz: Class[_ <: Actor], level: LogLevel, logName: String): ActorRef = {
|
||||
val name = "log" + Extension(system).id() + "-" + simpleName(clazz)
|
||||
val actor = system.systemActorOf(Props(clazz), name)
|
||||
|
|
@ -275,9 +276,9 @@ object LogSource {
|
|||
|
||||
// this one unfortunately does not work as implicit, because existential types have some weird behavior
|
||||
val fromClass: LogSource[Class[_]] = new LogSource[Class[_]] {
|
||||
def genString(c: Class[_]) = simpleName(c)
|
||||
override def genString(c: Class[_], system: ActorSystem) = simpleName(c) + "(" + system + ")"
|
||||
override def getClazz(c: Class[_]) = c
|
||||
def genString(c: Class[_]): String = Logging.simpleName(c)
|
||||
override def genString(c: Class[_], system: ActorSystem): String = genString(c) + "(" + system + ")"
|
||||
override def getClazz(c: Class[_]): Class[_] = c
|
||||
}
|
||||
implicit def fromAnyClass[T]: LogSource[Class[T]] = fromClass.asInstanceOf[LogSource[Class[T]]]
|
||||
|
||||
|
|
@ -310,7 +311,7 @@ object LogSource {
|
|||
case a: Actor ⇒ apply(a)
|
||||
case a: ActorRef ⇒ apply(a)
|
||||
case s: String ⇒ apply(s)
|
||||
case x ⇒ (simpleName(x), x.getClass)
|
||||
case x ⇒ (Logging.simpleName(x), x.getClass)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -324,7 +325,7 @@ object LogSource {
|
|||
case a: Actor ⇒ apply(a)
|
||||
case a: ActorRef ⇒ apply(a)
|
||||
case s: String ⇒ apply(s)
|
||||
case x ⇒ (simpleName(x) + "(" + system + ")", x.getClass)
|
||||
case x ⇒ (Logging.simpleName(x) + "(" + system + ")", x.getClass)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -363,9 +364,33 @@ object LogSource {
|
|||
*/
|
||||
object Logging {
|
||||
|
||||
object Extension extends ExtensionKey[LogExt]
|
||||
/**
|
||||
* Returns a 'safe' getSimpleName for the provided object's Class
|
||||
* @param obj
|
||||
* @return the simple name of the given object's Class
|
||||
*/
|
||||
def simpleName(obj: AnyRef): String = simpleName(obj.getClass)
|
||||
|
||||
class LogExt(system: ExtendedActorSystem) extends Extension {
|
||||
/**
|
||||
* Returns a 'safe' getSimpleName for the provided Class
|
||||
* @param obj
|
||||
* @return the simple name of the given Class
|
||||
*/
|
||||
def simpleName(clazz: Class[_]): String = {
|
||||
val n = clazz.getName
|
||||
val i = n.lastIndexOf('.')
|
||||
n.substring(i + 1)
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object Extension extends ExtensionKey[LogExt]
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class LogExt(system: ExtendedActorSystem) extends Extension {
|
||||
private val loggerId = new AtomicInteger
|
||||
def id() = loggerId.incrementAndGet()
|
||||
}
|
||||
|
|
@ -425,12 +450,6 @@ object Logging {
|
|||
// these type ascriptions/casts are necessary to avoid CCEs during construction while retaining correct type
|
||||
val AllLogLevels = Seq(ErrorLevel: AnyRef, WarningLevel, InfoLevel, DebugLevel).asInstanceOf[Seq[LogLevel]]
|
||||
|
||||
val errorFormat = "[ERROR] [%s] [%s] [%s] %s\n%s".intern
|
||||
val errorFormatWithoutCause = "[ERROR] [%s] [%s] [%s] %s".intern
|
||||
val warningFormat = "[WARN] [%s] [%s] [%s] %s".intern
|
||||
val infoFormat = "[INFO] [%s] [%s] [%s] %s".intern
|
||||
val debugFormat = "[DEBUG] [%s] [%s] [%s] %s".intern
|
||||
|
||||
/**
|
||||
* Obtain LoggingAdapter for the given actor system and source object. This
|
||||
* will use the system’s event stream and include the system’s address in the
|
||||
|
|
@ -511,7 +530,7 @@ object Logging {
|
|||
* Artificial exception injected into Error events if no Throwable is
|
||||
* supplied; used for getting a stack dump of error locations.
|
||||
*/
|
||||
class EventHandlerException extends AkkaException
|
||||
class EventHandlerException extends AkkaException("")
|
||||
|
||||
/**
|
||||
* Exception that wraps a LogEvent.
|
||||
|
|
@ -618,27 +637,34 @@ object Logging {
|
|||
// weird return type due to binary compatibility
|
||||
def loggerInitialized(): LoggerInitialized.type = LoggerInitialized
|
||||
|
||||
/**
|
||||
* LoggerInitializationException is thrown to indicate that there was a problem initializing a logger
|
||||
* @param msg
|
||||
*/
|
||||
class LoggerInitializationException(msg: String) extends AkkaException(msg)
|
||||
|
||||
trait StdOutLogger {
|
||||
import java.text.SimpleDateFormat
|
||||
import java.util.Date
|
||||
|
||||
val dateFormat = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss.S")
|
||||
private val dateFormat = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss.SSS")
|
||||
private val errorFormat = "[ERROR] [%s] [%s] [%s] %s\n%s".intern
|
||||
private val errorFormatWithoutCause = "[ERROR] [%s] [%s] [%s] %s".intern
|
||||
private val warningFormat = "[WARN] [%s] [%s] [%s] %s".intern
|
||||
private val infoFormat = "[INFO] [%s] [%s] [%s] %s".intern
|
||||
private val debugFormat = "[DEBUG] [%s] [%s] [%s] %s".intern
|
||||
|
||||
def timestamp = dateFormat.format(new Date)
|
||||
def timestamp(): String = synchronized { dateFormat.format(new Date) } // SDF isn't threadsafe
|
||||
|
||||
def print(event: Any) {
|
||||
event match {
|
||||
case e: Error ⇒ error(e)
|
||||
case e: Warning ⇒ warning(e)
|
||||
case e: Info ⇒ info(e)
|
||||
case e: Debug ⇒ debug(e)
|
||||
case e ⇒ warning(Warning(simpleName(this), this.getClass, "received unexpected event of class " + e.getClass + ": " + e))
|
||||
}
|
||||
def print(event: Any): Unit = event match {
|
||||
case e: Error ⇒ error(e)
|
||||
case e: Warning ⇒ warning(e)
|
||||
case e: Info ⇒ info(e)
|
||||
case e: Debug ⇒ debug(e)
|
||||
case e ⇒ warning(Warning(simpleName(this), this.getClass, "received unexpected event of class " + e.getClass + ": " + e))
|
||||
}
|
||||
|
||||
def error(event: Error) = {
|
||||
def error(event: Error): Unit = {
|
||||
val f = if (event.cause == Error.NoCause) errorFormatWithoutCause else errorFormat
|
||||
println(f.format(
|
||||
timestamp,
|
||||
|
|
@ -648,21 +674,21 @@ object Logging {
|
|||
stackTraceFor(event.cause)))
|
||||
}
|
||||
|
||||
def warning(event: Warning) =
|
||||
def warning(event: Warning): Unit =
|
||||
println(warningFormat.format(
|
||||
timestamp,
|
||||
event.thread.getName,
|
||||
event.logSource,
|
||||
event.message))
|
||||
|
||||
def info(event: Info) =
|
||||
def info(event: Info): Unit =
|
||||
println(infoFormat.format(
|
||||
timestamp,
|
||||
event.thread.getName,
|
||||
event.logSource,
|
||||
event.message))
|
||||
|
||||
def debug(event: Debug) =
|
||||
def debug(event: Debug): Unit =
|
||||
println(debugFormat.format(
|
||||
timestamp,
|
||||
event.thread.getName,
|
||||
|
|
@ -683,8 +709,8 @@ object Logging {
|
|||
override val toString = "StandardOutLogger"
|
||||
override def !(message: Any)(implicit sender: ActorRef = null): Unit = print(message)
|
||||
}
|
||||
|
||||
val StandardOutLogger = new StandardOutLogger
|
||||
val StandardOutLoggerName = StandardOutLogger.getClass.getName
|
||||
|
||||
/**
|
||||
* Actor wrapper around the standard output logger. If
|
||||
|
|
@ -702,7 +728,7 @@ object Logging {
|
|||
* Returns the StackTrace for the given Throwable as a String
|
||||
*/
|
||||
def stackTraceFor(e: Throwable): String = e match {
|
||||
case null | Error.NoCause ⇒ ""
|
||||
case null | Error.NoCause | _: NoStackTrace ⇒ ""
|
||||
case other ⇒
|
||||
val sw = new java.io.StringWriter
|
||||
val pw = new java.io.PrintWriter(sw)
|
||||
|
|
@ -746,51 +772,51 @@ trait LoggingAdapter {
|
|||
* These actually implement the passing on of the messages to be logged.
|
||||
* Will not be called if is...Enabled returned false.
|
||||
*/
|
||||
protected def notifyError(message: String)
|
||||
protected def notifyError(cause: Throwable, message: String)
|
||||
protected def notifyWarning(message: String)
|
||||
protected def notifyInfo(message: String)
|
||||
protected def notifyDebug(message: String)
|
||||
protected def notifyError(message: String): Unit
|
||||
protected def notifyError(cause: Throwable, message: String): Unit
|
||||
protected def notifyWarning(message: String): Unit
|
||||
protected def notifyInfo(message: String): Unit
|
||||
protected def notifyDebug(message: String): Unit
|
||||
|
||||
/*
|
||||
* The rest is just the widening of the API for the user's convenience.
|
||||
*/
|
||||
|
||||
def error(cause: Throwable, message: String) { if (isErrorEnabled) notifyError(cause, message) }
|
||||
def error(cause: Throwable, template: String, arg1: Any) { if (isErrorEnabled) notifyError(cause, format1(template, arg1)) }
|
||||
def error(cause: Throwable, template: String, arg1: Any, arg2: Any) { if (isErrorEnabled) notifyError(cause, format(template, arg1, arg2)) }
|
||||
def error(cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any) { if (isErrorEnabled) notifyError(cause, format(template, arg1, arg2, arg3)) }
|
||||
def error(cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) { if (isErrorEnabled) notifyError(cause, format(template, arg1, arg2, arg3, arg4)) }
|
||||
def error(cause: Throwable, message: String): Unit = { if (isErrorEnabled) notifyError(cause, message) }
|
||||
def error(cause: Throwable, template: String, arg1: Any): Unit = { if (isErrorEnabled) notifyError(cause, format1(template, arg1)) }
|
||||
def error(cause: Throwable, template: String, arg1: Any, arg2: Any): Unit = { if (isErrorEnabled) notifyError(cause, format(template, arg1, arg2)) }
|
||||
def error(cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any): Unit = { if (isErrorEnabled) notifyError(cause, format(template, arg1, arg2, arg3)) }
|
||||
def error(cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = { if (isErrorEnabled) notifyError(cause, format(template, arg1, arg2, arg3, arg4)) }
|
||||
|
||||
def error(message: String) { if (isErrorEnabled) notifyError(message) }
|
||||
def error(template: String, arg1: Any) { if (isErrorEnabled) notifyError(format1(template, arg1)) }
|
||||
def error(template: String, arg1: Any, arg2: Any) { if (isErrorEnabled) notifyError(format(template, arg1, arg2)) }
|
||||
def error(template: String, arg1: Any, arg2: Any, arg3: Any) { if (isErrorEnabled) notifyError(format(template, arg1, arg2, arg3)) }
|
||||
def error(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) { if (isErrorEnabled) notifyError(format(template, arg1, arg2, arg3, arg4)) }
|
||||
def error(message: String): Unit = { if (isErrorEnabled) notifyError(message) }
|
||||
def error(template: String, arg1: Any): Unit = { if (isErrorEnabled) notifyError(format1(template, arg1)) }
|
||||
def error(template: String, arg1: Any, arg2: Any): Unit = { if (isErrorEnabled) notifyError(format(template, arg1, arg2)) }
|
||||
def error(template: String, arg1: Any, arg2: Any, arg3: Any): Unit = { if (isErrorEnabled) notifyError(format(template, arg1, arg2, arg3)) }
|
||||
def error(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = { if (isErrorEnabled) notifyError(format(template, arg1, arg2, arg3, arg4)) }
|
||||
|
||||
def warning(message: String) { if (isWarningEnabled) notifyWarning(message) }
|
||||
def warning(template: String, arg1: Any) { if (isWarningEnabled) notifyWarning(format1(template, arg1)) }
|
||||
def warning(template: String, arg1: Any, arg2: Any) { if (isWarningEnabled) notifyWarning(format(template, arg1, arg2)) }
|
||||
def warning(template: String, arg1: Any, arg2: Any, arg3: Any) { if (isWarningEnabled) notifyWarning(format(template, arg1, arg2, arg3)) }
|
||||
def warning(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) { if (isWarningEnabled) notifyWarning(format(template, arg1, arg2, arg3, arg4)) }
|
||||
def warning(message: String): Unit = { if (isWarningEnabled) notifyWarning(message) }
|
||||
def warning(template: String, arg1: Any): Unit = { if (isWarningEnabled) notifyWarning(format1(template, arg1)) }
|
||||
def warning(template: String, arg1: Any, arg2: Any): Unit = { if (isWarningEnabled) notifyWarning(format(template, arg1, arg2)) }
|
||||
def warning(template: String, arg1: Any, arg2: Any, arg3: Any): Unit = { if (isWarningEnabled) notifyWarning(format(template, arg1, arg2, arg3)) }
|
||||
def warning(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = { if (isWarningEnabled) notifyWarning(format(template, arg1, arg2, arg3, arg4)) }
|
||||
|
||||
def info(message: String) { if (isInfoEnabled) notifyInfo(message) }
|
||||
def info(template: String, arg1: Any) { if (isInfoEnabled) notifyInfo(format1(template, arg1)) }
|
||||
def info(template: String, arg1: Any, arg2: Any) { if (isInfoEnabled) notifyInfo(format(template, arg1, arg2)) }
|
||||
def info(template: String, arg1: Any, arg2: Any, arg3: Any) { if (isInfoEnabled) notifyInfo(format(template, arg1, arg2, arg3)) }
|
||||
def info(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) { if (isInfoEnabled) notifyInfo(format(template, arg1, arg2, arg3, arg4)) }
|
||||
def info(template: String, arg1: Any): Unit = { if (isInfoEnabled) notifyInfo(format1(template, arg1)) }
|
||||
def info(template: String, arg1: Any, arg2: Any): Unit = { if (isInfoEnabled) notifyInfo(format(template, arg1, arg2)) }
|
||||
def info(template: String, arg1: Any, arg2: Any, arg3: Any): Unit = { if (isInfoEnabled) notifyInfo(format(template, arg1, arg2, arg3)) }
|
||||
def info(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = { if (isInfoEnabled) notifyInfo(format(template, arg1, arg2, arg3, arg4)) }
|
||||
|
||||
def debug(message: String) { if (isDebugEnabled) notifyDebug(message) }
|
||||
def debug(template: String, arg1: Any) { if (isDebugEnabled) notifyDebug(format1(template, arg1)) }
|
||||
def debug(template: String, arg1: Any, arg2: Any) { if (isDebugEnabled) notifyDebug(format(template, arg1, arg2)) }
|
||||
def debug(template: String, arg1: Any, arg2: Any, arg3: Any) { if (isDebugEnabled) notifyDebug(format(template, arg1, arg2, arg3)) }
|
||||
def debug(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) { if (isDebugEnabled) notifyDebug(format(template, arg1, arg2, arg3, arg4)) }
|
||||
def debug(template: String, arg1: Any): Unit = { if (isDebugEnabled) notifyDebug(format1(template, arg1)) }
|
||||
def debug(template: String, arg1: Any, arg2: Any): Unit = { if (isDebugEnabled) notifyDebug(format(template, arg1, arg2)) }
|
||||
def debug(template: String, arg1: Any, arg2: Any, arg3: Any): Unit = { if (isDebugEnabled) notifyDebug(format(template, arg1, arg2, arg3)) }
|
||||
def debug(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = { if (isDebugEnabled) notifyDebug(format(template, arg1, arg2, arg3, arg4)) }
|
||||
|
||||
def log(level: Logging.LogLevel, message: String) { if (isEnabled(level)) notifyLog(level, message) }
|
||||
def log(level: Logging.LogLevel, template: String, arg1: Any) { if (isEnabled(level)) notifyLog(level, format1(template, arg1)) }
|
||||
def log(level: Logging.LogLevel, template: String, arg1: Any, arg2: Any) { if (isEnabled(level)) notifyLog(level, format(template, arg1, arg2)) }
|
||||
def log(level: Logging.LogLevel, template: String, arg1: Any, arg2: Any, arg3: Any) { if (isEnabled(level)) notifyLog(level, format(template, arg1, arg2, arg3)) }
|
||||
def log(level: Logging.LogLevel, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any) { if (isEnabled(level)) notifyLog(level, format(template, arg1, arg2, arg3, arg4)) }
|
||||
def log(level: Logging.LogLevel, template: String, arg1: Any): Unit = { if (isEnabled(level)) notifyLog(level, format1(template, arg1)) }
|
||||
def log(level: Logging.LogLevel, template: String, arg1: Any, arg2: Any): Unit = { if (isEnabled(level)) notifyLog(level, format(template, arg1, arg2)) }
|
||||
def log(level: Logging.LogLevel, template: String, arg1: Any, arg2: Any, arg3: Any): Unit = { if (isEnabled(level)) notifyLog(level, format(template, arg1, arg2, arg3)) }
|
||||
def log(level: Logging.LogLevel, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = { if (isEnabled(level)) notifyLog(level, format(template, arg1, arg2, arg3, arg4)) }
|
||||
|
||||
final def isEnabled(level: Logging.LogLevel): Boolean = level match {
|
||||
case Logging.ErrorLevel ⇒ isErrorEnabled
|
||||
|
|
@ -806,14 +832,14 @@ trait LoggingAdapter {
|
|||
case Logging.DebugLevel ⇒ if (isDebugEnabled) notifyDebug(message)
|
||||
}
|
||||
|
||||
private def format1(t: String, arg: Any) = arg match {
|
||||
private def format1(t: String, arg: Any): String = arg match {
|
||||
case a: Array[_] if !a.getClass.getComponentType.isPrimitive ⇒ format(t, a: _*)
|
||||
case a: Array[_] ⇒ format(t, (a map (_.asInstanceOf[AnyRef]): _*))
|
||||
case x ⇒ format(t, x)
|
||||
}
|
||||
|
||||
def format(t: String, arg: Any*) = {
|
||||
val sb = new StringBuilder
|
||||
def format(t: String, arg: Any*): String = {
|
||||
val sb = new StringBuilder //FIXME add some decent size hint here
|
||||
var p = 0
|
||||
var rest = t
|
||||
while (p < arg.length) {
|
||||
|
|
@ -823,17 +849,15 @@ trait LoggingAdapter {
|
|||
rest = ""
|
||||
p = arg.length
|
||||
} else {
|
||||
sb.append(rest.substring(0, index))
|
||||
sb.append(arg(p))
|
||||
sb.append(rest.substring(0, index)).append(arg(p))
|
||||
rest = rest.substring(index + 2)
|
||||
p += 1
|
||||
}
|
||||
}
|
||||
sb.append(rest)
|
||||
sb.toString
|
||||
sb.append(rest).toString
|
||||
}
|
||||
}
|
||||
|
||||
//FIXME DOCUMENT
|
||||
class BusLogging(val bus: LoggingBus, val logSource: String, val logClass: Class[_]) extends LoggingAdapter {
|
||||
|
||||
import Logging._
|
||||
|
|
@ -843,14 +867,9 @@ class BusLogging(val bus: LoggingBus, val logSource: String, val logClass: Class
|
|||
def isInfoEnabled = bus.logLevel >= InfoLevel
|
||||
def isDebugEnabled = bus.logLevel >= DebugLevel
|
||||
|
||||
protected def notifyError(message: String) { bus.publish(Error(logSource, logClass, message)) }
|
||||
|
||||
protected def notifyError(cause: Throwable, message: String) { bus.publish(Error(cause, logSource, logClass, message)) }
|
||||
|
||||
protected def notifyWarning(message: String) { bus.publish(Warning(logSource, logClass, message)) }
|
||||
|
||||
protected def notifyInfo(message: String) { bus.publish(Info(logSource, logClass, message)) }
|
||||
|
||||
protected def notifyDebug(message: String) { bus.publish(Debug(logSource, logClass, message)) }
|
||||
|
||||
protected def notifyError(message: String): Unit = bus.publish(Error(logSource, logClass, message))
|
||||
protected def notifyError(cause: Throwable, message: String): Unit = bus.publish(Error(cause, logSource, logClass, message))
|
||||
protected def notifyWarning(message: String): Unit = bus.publish(Warning(logSource, logClass, message))
|
||||
protected def notifyInfo(message: String): Unit = bus.publish(Info(logSource, logClass, message))
|
||||
protected def notifyDebug(message: String): Unit = bus.publish(Debug(logSource, logClass, message))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -26,9 +26,7 @@ object LoggingReceive {
|
|||
*/
|
||||
def apply(r: Receive)(implicit context: ActorContext): Receive = r match {
|
||||
case _: LoggingReceive ⇒ r
|
||||
case _ ⇒
|
||||
if (context.system.settings.AddLoggingReceive) new LoggingReceive(None, r)
|
||||
else r
|
||||
case _ ⇒ if (context.system.settings.AddLoggingReceive) new LoggingReceive(None, r) else r
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -37,7 +35,7 @@ object LoggingReceive {
|
|||
* @param source the log source, if not defined the actor of the context will be used
|
||||
*/
|
||||
class LoggingReceive(source: Option[AnyRef], r: Receive)(implicit context: ActorContext) extends Receive {
|
||||
def isDefinedAt(o: Any) = {
|
||||
def isDefinedAt(o: Any): Boolean = {
|
||||
val handled = r.isDefinedAt(o)
|
||||
val (str, clazz) = LogSource.fromAnyRef(source getOrElse context.asInstanceOf[ActorCell].actor)
|
||||
context.system.eventStream.publish(Debug(str, clazz, "received " + (if (handled) "handled" else "unhandled") + " message " + o))
|
||||
|
|
|
|||
|
|
@ -1,19 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka
|
||||
|
||||
import annotation.target._
|
||||
|
||||
/**
|
||||
* This annotation marks a feature which is not yet considered stable and may
|
||||
* change or be removed in a future release.
|
||||
*
|
||||
* @since 1.2
|
||||
*/
|
||||
@getter
|
||||
@setter
|
||||
@beanGetter
|
||||
@beanSetter
|
||||
final class experimental(since: String) extends annotation.StaticAnnotation
|
||||
|
|
@ -24,28 +24,14 @@ trait Function2[T1, T2, R] {
|
|||
* A Procedure is like a Function, but it doesn't produce a return value.
|
||||
*/
|
||||
trait Procedure[T] {
|
||||
def apply(param: T)
|
||||
}
|
||||
|
||||
/**
|
||||
* A Procedure is like a Function, but it doesn't produce a return value.
|
||||
*/
|
||||
trait Procedure2[T1, T2] {
|
||||
def apply(param: T1, param2: T2)
|
||||
}
|
||||
|
||||
/**
|
||||
* An executable piece of code that takes no parameters and doesn't return any value.
|
||||
*/
|
||||
trait SideEffect {
|
||||
def apply()
|
||||
def apply(param: T): Unit
|
||||
}
|
||||
|
||||
/**
|
||||
* An executable piece of code that takes no parameters and doesn't return any value.
|
||||
*/
|
||||
trait Effect {
|
||||
def apply()
|
||||
def apply(): Unit
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -67,9 +53,9 @@ sealed abstract class Option[A] extends java.lang.Iterable[A] {
|
|||
|
||||
def get: A
|
||||
def isEmpty: Boolean
|
||||
def isDefined = !isEmpty
|
||||
def isDefined: Boolean = !isEmpty
|
||||
def asScala: scala.Option[A]
|
||||
def iterator = if (isEmpty) Iterator.empty else Iterator.single(get)
|
||||
def iterator: java.util.Iterator[A] = if (isEmpty) Iterator.empty else Iterator.single(get)
|
||||
}
|
||||
|
||||
object Option {
|
||||
|
|
@ -102,18 +88,18 @@ object Option {
|
|||
* <code>A</code>.
|
||||
*/
|
||||
final case class Some[A](v: A) extends Option[A] {
|
||||
def get = v
|
||||
def isEmpty = false
|
||||
def asScala = scala.Some(v)
|
||||
def get: A = v
|
||||
def isEmpty: Boolean = false
|
||||
def asScala: scala.Some[A] = scala.Some(v)
|
||||
}
|
||||
|
||||
/**
|
||||
* This case object represents non-existent values.
|
||||
*/
|
||||
private case object None extends Option[Nothing] {
|
||||
def get = throw new NoSuchElementException("None.get")
|
||||
def isEmpty = true
|
||||
def asScala = scala.None
|
||||
def get: Nothing = throw new NoSuchElementException("None.get")
|
||||
def isEmpty: Boolean = true
|
||||
def asScala: scala.None.type = scala.None
|
||||
}
|
||||
|
||||
implicit def java2ScalaOption[A](o: Option[A]): scala.Option[A] = o.asScala
|
||||
|
|
|
|||
|
|
@ -46,7 +46,7 @@ trait AskSupport {
|
|||
* Sends a message asynchronously and returns a [[akka.dispatch.Future]]
|
||||
* holding the eventual reply message; this means that the target actor
|
||||
* needs to send the result to the `sender` reference provided. The Future
|
||||
* will be completed with an [[akka.actor.AskTimeoutException]] after the
|
||||
* will be completed with an [[akka.pattern.AskTimeoutException]] after the
|
||||
* given timeout has expired; this is independent from any timeout applied
|
||||
* while awaiting a result for this future (i.e. in
|
||||
* `Await.result(..., timeout)`).
|
||||
|
|
@ -96,7 +96,7 @@ trait AskSupport {
|
|||
* Sends a message asynchronously and returns a [[akka.dispatch.Future]]
|
||||
* holding the eventual reply message; this means that the target actor
|
||||
* needs to send the result to the `sender` reference provided. The Future
|
||||
* will be completed with an [[akka.actor.AskTimeoutException]] after the
|
||||
* will be completed with an [[akka.pattern.AskTimeoutException]] after the
|
||||
* given timeout has expired; this is independent from any timeout applied
|
||||
* while awaiting a result for this future (i.e. in
|
||||
* `Await.result(..., timeout)`).
|
||||
|
|
@ -126,7 +126,7 @@ trait AskSupport {
|
|||
* Sends a message asynchronously and returns a [[akka.dispatch.Future]]
|
||||
* holding the eventual reply message; this means that the target actor
|
||||
* needs to send the result to the `sender` reference provided. The Future
|
||||
* will be completed with an [[akka.actor.AskTimeoutException]] after the
|
||||
* will be completed with an [[akka.pattern.AskTimeoutException]] after the
|
||||
* given timeout has expired; this is independent from any timeout applied
|
||||
* while awaiting a result for this future (i.e. in
|
||||
* `Await.result(..., timeout)`).
|
||||
|
|
@ -157,6 +157,8 @@ trait AskSupport {
|
|||
/**
|
||||
* Akka private optimized representation of the temporary actor spawned to
|
||||
* receive the reply to an "ask" operation.
|
||||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final class PromiseActorRef private (val provider: ActorRefProvider, val result: Promise[Any])
|
||||
extends MinimalActorRef {
|
||||
|
|
@ -182,14 +184,12 @@ private[akka] final class PromiseActorRef private (val provider: ActorRefProvide
|
|||
private def state: AnyRef = Unsafe.instance.getObjectVolatile(this, stateOffset)
|
||||
|
||||
@inline
|
||||
private def updateState(oldState: AnyRef, newState: AnyRef): Boolean =
|
||||
Unsafe.instance.compareAndSwapObject(this, stateOffset, oldState, newState)
|
||||
private def updateState(oldState: AnyRef, newState: AnyRef): Boolean = Unsafe.instance.compareAndSwapObject(this, stateOffset, oldState, newState)
|
||||
|
||||
@inline
|
||||
private def setState(newState: AnyRef): Unit =
|
||||
Unsafe.instance.putObjectVolatile(this, stateOffset, newState)
|
||||
private def setState(newState: AnyRef): Unit = Unsafe.instance.putObjectVolatile(this, stateOffset, newState)
|
||||
|
||||
override def getParent = provider.tempContainer
|
||||
override def getParent: InternalActorRef = provider.tempContainer
|
||||
|
||||
/**
|
||||
* Contract of this method:
|
||||
|
|
@ -234,7 +234,7 @@ private[akka] final class PromiseActorRef private (val provider: ActorRefProvide
|
|||
case _ ⇒
|
||||
}
|
||||
|
||||
override def isTerminated = state match {
|
||||
override def isTerminated: Boolean = state match {
|
||||
case Stopped | _: StoppedWithPath ⇒ true
|
||||
case _ ⇒ false
|
||||
}
|
||||
|
|
@ -263,6 +263,9 @@ private[akka] final class PromiseActorRef private (val provider: ActorRefProvide
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object PromiseActorRef {
|
||||
private case object Registering
|
||||
private case object Stopped
|
||||
|
|
@ -272,9 +275,7 @@ private[akka] object PromiseActorRef {
|
|||
val result = Promise[Any]()(provider.dispatcher)
|
||||
val a = new PromiseActorRef(provider, result)
|
||||
val f = provider.scheduler.scheduleOnce(timeout.duration) { result.tryComplete(Left(new AskTimeoutException("Timed out"))) }
|
||||
result onComplete { _ ⇒
|
||||
try a.stop() finally f.cancel()
|
||||
}
|
||||
result onComplete { _ ⇒ try a.stop() finally f.cancel() }
|
||||
a
|
||||
}
|
||||
}
|
||||
|
|
@ -4,9 +4,9 @@
|
|||
|
||||
package akka.pattern
|
||||
|
||||
import akka.actor.{ ActorRef, Actor, ActorSystem, Props, PoisonPill, Terminated, ReceiveTimeout, ActorTimeoutException }
|
||||
import akka.dispatch.{ Promise, Future }
|
||||
import akka.util.Duration
|
||||
import akka.actor._
|
||||
import akka.util.{ Timeout, Duration }
|
||||
|
||||
trait GracefulStopSupport {
|
||||
/**
|
||||
|
|
@ -14,34 +14,39 @@ trait GracefulStopSupport {
|
|||
* existing messages of the target actor has been processed and the actor has been
|
||||
* terminated.
|
||||
*
|
||||
* Useful when you need to wait for termination or compose ordered termination of several actors.
|
||||
* Useful when you need to wait for termination or compose ordered termination of several actors,
|
||||
* which should only be done outside of the ActorSystem as blocking inside Actors is discouraged.
|
||||
*
|
||||
* <b>IMPORTANT NOTICE:</b> the actor being terminated and its supervisor
|
||||
* being informed of the availability of the deceased actor’s name are two
|
||||
* distinct operations, which do not obey any reliable ordering. Especially
|
||||
* the following will NOT work:
|
||||
*
|
||||
* {{{
|
||||
* def receive = {
|
||||
* case msg =>
|
||||
* Await.result(gracefulStop(someChild, timeout), timeout)
|
||||
* context.actorOf(Props(...), "someChild") // assuming that that was someChild’s name, this will NOT work
|
||||
* }
|
||||
* }}}
|
||||
*
|
||||
* If the target actor isn't terminated within the timeout the [[akka.dispatch.Future]]
|
||||
* is completed with failure [[akka.actor.ActorTimeoutException]].
|
||||
* is completed with failure [[akka.pattern.AskTimeoutException]].
|
||||
*/
|
||||
def gracefulStop(target: ActorRef, timeout: Duration)(implicit system: ActorSystem): Future[Boolean] = {
|
||||
if (target.isTerminated) {
|
||||
Promise.successful(true)
|
||||
} else {
|
||||
val result = Promise[Boolean]()
|
||||
system.actorOf(Props(new Actor {
|
||||
// Terminated will be received when target has been stopped
|
||||
context watch target
|
||||
} else system match {
|
||||
case e: ExtendedActorSystem ⇒
|
||||
val ref = PromiseActorRef(e.provider, Timeout(timeout))
|
||||
e.deathWatch.subscribe(ref, target)
|
||||
ref.result onComplete {
|
||||
case Right(Terminated(`target`)) ⇒ () // Ignore
|
||||
case _ ⇒ e.deathWatch.unsubscribe(ref, target)
|
||||
} // Just making sure we're not leaking here
|
||||
target ! PoisonPill
|
||||
// ReceiveTimeout will be received if nothing else is received within the timeout
|
||||
context setReceiveTimeout timeout
|
||||
|
||||
def receive = {
|
||||
case Terminated(a) if a == target ⇒
|
||||
result success true
|
||||
context stop self
|
||||
case ReceiveTimeout ⇒
|
||||
result failure new ActorTimeoutException(
|
||||
"Failed to stop [%s] within [%s]".format(target.path, context.receiveTimeout))
|
||||
context stop self
|
||||
}
|
||||
}))
|
||||
result
|
||||
ref.result map { case Terminated(`target`) ⇒ true }
|
||||
case s ⇒ throw new IllegalArgumentException("Unknown ActorSystem implementation: '" + s + "'")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -18,7 +18,7 @@ object Patterns {
|
|||
* Sends a message asynchronously and returns a [[akka.dispatch.Future]]
|
||||
* holding the eventual reply message; this means that the target actor
|
||||
* needs to send the result to the `sender` reference provided. The Future
|
||||
* will be completed with an [[akka.actor.AskTimeoutException]] after the
|
||||
* will be completed with an [[akka.pattern.AskTimeoutException]] after the
|
||||
* given timeout has expired; this is independent from any timeout applied
|
||||
* while awaiting a result for this future (i.e. in
|
||||
* `Await.result(..., timeout)`).
|
||||
|
|
@ -49,7 +49,7 @@ object Patterns {
|
|||
* Sends a message asynchronously and returns a [[akka.dispatch.Future]]
|
||||
* holding the eventual reply message; this means that the target actor
|
||||
* needs to send the result to the `sender` reference provided. The Future
|
||||
* will be completed with an [[akka.actor.AskTimeoutException]] after the
|
||||
* will be completed with an [[akka.pattern.AskTimeoutException]] after the
|
||||
* given timeout has expired; this is independent from any timeout applied
|
||||
* while awaiting a result for this future (i.e. in
|
||||
* `Await.result(..., timeout)`).
|
||||
|
|
@ -100,7 +100,7 @@ object Patterns {
|
|||
* Useful when you need to wait for termination or compose ordered termination of several actors.
|
||||
*
|
||||
* If the target actor isn't terminated within the timeout the [[akka.dispatch.Future]]
|
||||
* is completed with failure [[akka.actor.ActorTimeoutException]].
|
||||
* is completed with failure [[akka.pattern.AskTimeoutException]].
|
||||
*/
|
||||
def gracefulStop(target: ActorRef, timeout: Duration, system: ActorSystem): Future[java.lang.Boolean] =
|
||||
scalaGracefulStop(target, timeout)(system).asInstanceOf[Future[java.lang.Boolean]]
|
||||
|
|
|
|||
|
|
@ -10,10 +10,8 @@ import akka.actor._
|
|||
* An Iterable that also contains a version.
|
||||
*/
|
||||
trait VersionedIterable[A] {
|
||||
val version: Long
|
||||
|
||||
def version: Long
|
||||
def iterable: Iterable[A]
|
||||
|
||||
def apply(): Iterable[A] = iterable
|
||||
}
|
||||
|
||||
|
|
@ -42,7 +40,7 @@ trait ConnectionManager {
|
|||
/**
|
||||
* Shuts the connection manager down, which stops all managed actors
|
||||
*/
|
||||
def shutdown()
|
||||
def shutdown(): Unit
|
||||
|
||||
/**
|
||||
* Returns a VersionedIterator containing all connected ActorRefs at some moment in time. Since there is
|
||||
|
|
@ -59,5 +57,5 @@ trait ConnectionManager {
|
|||
*
|
||||
* @param ref the dead
|
||||
*/
|
||||
def remove(deadRef: ActorRef)
|
||||
def remove(deadRef: ActorRef): Unit
|
||||
}
|
||||
|
|
|
|||
|
|
@ -23,7 +23,7 @@ class ConsistentHash[T](nodes: Seq[T], replicas: Int) {
|
|||
|
||||
nodes.foreach(this += _)
|
||||
|
||||
def +=(node: T) {
|
||||
def +=(node: T): Unit = {
|
||||
cluster += node
|
||||
(1 to replicas) foreach { replica ⇒
|
||||
val key = hashFor((node + ":" + replica).getBytes("UTF-8"))
|
||||
|
|
@ -32,7 +32,7 @@ class ConsistentHash[T](nodes: Seq[T], replicas: Int) {
|
|||
}
|
||||
}
|
||||
|
||||
def -=(node: T) {
|
||||
def -=(node: T): Unit = {
|
||||
cluster -= node
|
||||
(1 to replicas) foreach { replica ⇒
|
||||
val key = hashFor((node + ":" + replica).getBytes("UTF-8"))
|
||||
|
|
@ -96,7 +96,7 @@ class MurmurHash[@specialized(Int, Long, Float, Double) T](seed: Int) extends (T
|
|||
private var hashvalue = h
|
||||
|
||||
/** Begin a new hash using the same seed. */
|
||||
def reset() {
|
||||
def reset(): Unit = {
|
||||
h = startHash(seed)
|
||||
c = hiddenMagicA
|
||||
k = hiddenMagicB
|
||||
|
|
@ -104,7 +104,7 @@ class MurmurHash[@specialized(Int, Long, Float, Double) T](seed: Int) extends (T
|
|||
}
|
||||
|
||||
/** Incorporate the hash value of one item. */
|
||||
def apply(t: T) {
|
||||
def apply(t: T): Unit = {
|
||||
h = extendHash(h, t.##, c, k)
|
||||
c = nextMagicA(c)
|
||||
k = nextMagicB(k)
|
||||
|
|
@ -112,7 +112,7 @@ class MurmurHash[@specialized(Int, Long, Float, Double) T](seed: Int) extends (T
|
|||
}
|
||||
|
||||
/** Incorporate a known hash value. */
|
||||
def append(i: Int) {
|
||||
def append(i: Int): Unit = {
|
||||
h = extendHash(h, i, c, k)
|
||||
c = nextMagicA(c)
|
||||
k = nextMagicB(k)
|
||||
|
|
@ -120,14 +120,15 @@ class MurmurHash[@specialized(Int, Long, Float, Double) T](seed: Int) extends (T
|
|||
}
|
||||
|
||||
/** Retrieve the hash value */
|
||||
def hash = {
|
||||
def hash: Int = {
|
||||
if (!hashed) {
|
||||
hashvalue = finalizeHash(h)
|
||||
hashed = true
|
||||
}
|
||||
hashvalue
|
||||
}
|
||||
override def hashCode = hash
|
||||
|
||||
override def hashCode: Int = hash
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -143,35 +144,35 @@ class MurmurHash[@specialized(Int, Long, Float, Double) T](seed: Int) extends (T
|
|||
object MurmurHash {
|
||||
// Magic values used for MurmurHash's 32 bit hash.
|
||||
// Don't change these without consulting a hashing expert!
|
||||
final private val visibleMagic = 0x971e137b
|
||||
final private val hiddenMagicA = 0x95543787
|
||||
final private val hiddenMagicB = 0x2ad7eb25
|
||||
final private val visibleMixer = 0x52dce729
|
||||
final private val hiddenMixerA = 0x7b7d159c
|
||||
final private val hiddenMixerB = 0x6bce6396
|
||||
final private val finalMixer1 = 0x85ebca6b
|
||||
final private val finalMixer2 = 0xc2b2ae35
|
||||
final private val visibleMagic: Int = 0x971e137b
|
||||
final private val hiddenMagicA: Int = 0x95543787
|
||||
final private val hiddenMagicB: Int = 0x2ad7eb25
|
||||
final private val visibleMixer: Int = 0x52dce729
|
||||
final private val hiddenMixerA: Int = 0x7b7d159c
|
||||
final private val hiddenMixerB: Int = 0x6bce6396
|
||||
final private val finalMixer1: Int = 0x85ebca6b
|
||||
final private val finalMixer2: Int = 0xc2b2ae35
|
||||
|
||||
// Arbitrary values used for hashing certain classes
|
||||
final private val seedString = 0xf7ca7fd2
|
||||
final private val seedArray = 0x3c074a61
|
||||
final private val seedString: Int = 0xf7ca7fd2
|
||||
final private val seedArray: Int = 0x3c074a61
|
||||
|
||||
/** The first 23 magic integers from the first stream are stored here */
|
||||
val storedMagicA =
|
||||
val storedMagicA: Array[Int] =
|
||||
Iterator.iterate(hiddenMagicA)(nextMagicA).take(23).toArray
|
||||
|
||||
/** The first 23 magic integers from the second stream are stored here */
|
||||
val storedMagicB =
|
||||
val storedMagicB: Array[Int] =
|
||||
Iterator.iterate(hiddenMagicB)(nextMagicB).take(23).toArray
|
||||
|
||||
/** Begin a new hash with a seed value. */
|
||||
def startHash(seed: Int) = seed ^ visibleMagic
|
||||
def startHash(seed: Int): Int = seed ^ visibleMagic
|
||||
|
||||
/** The initial magic integers in the first stream. */
|
||||
def startMagicA = hiddenMagicA
|
||||
def startMagicA: Int = hiddenMagicA
|
||||
|
||||
/** The initial magic integer in the second stream. */
|
||||
def startMagicB = hiddenMagicB
|
||||
def startMagicB: Int = hiddenMagicB
|
||||
|
||||
/**
|
||||
* Incorporates a new value into an existing hash.
|
||||
|
|
@ -182,18 +183,17 @@ object MurmurHash {
|
|||
* @param magicB a magic integer from a different stream
|
||||
* @return the updated hash value
|
||||
*/
|
||||
def extendHash(hash: Int, value: Int, magicA: Int, magicB: Int) = {
|
||||
def extendHash(hash: Int, value: Int, magicA: Int, magicB: Int): Int =
|
||||
(hash ^ rotl(value * magicA, 11) * magicB) * 3 + visibleMixer
|
||||
}
|
||||
|
||||
/** Given a magic integer from the first stream, compute the next */
|
||||
def nextMagicA(magicA: Int) = magicA * 5 + hiddenMixerA
|
||||
def nextMagicA(magicA: Int): Int = magicA * 5 + hiddenMixerA
|
||||
|
||||
/** Given a magic integer from the second stream, compute the next */
|
||||
def nextMagicB(magicB: Int) = magicB * 5 + hiddenMixerB
|
||||
def nextMagicB(magicB: Int): Int = magicB * 5 + hiddenMixerB
|
||||
|
||||
/** Once all hashes have been incorporated, this performs a final mixing */
|
||||
def finalizeHash(hash: Int) = {
|
||||
def finalizeHash(hash: Int): Int = {
|
||||
var i = (hash ^ (hash >>> 16))
|
||||
i *= finalMixer1
|
||||
i ^= (i >>> 13)
|
||||
|
|
@ -203,7 +203,7 @@ object MurmurHash {
|
|||
}
|
||||
|
||||
/** Compute a high-quality hash of an array */
|
||||
def arrayHash[@specialized T](a: Array[T]) = {
|
||||
def arrayHash[@specialized T](a: Array[T]): Int = {
|
||||
var h = startHash(a.length * seedArray)
|
||||
var c = hiddenMagicA
|
||||
var k = hiddenMagicB
|
||||
|
|
@ -218,7 +218,7 @@ object MurmurHash {
|
|||
}
|
||||
|
||||
/** Compute a high-quality hash of a string */
|
||||
def stringHash(s: String) = {
|
||||
def stringHash(s: String): Int = {
|
||||
var h = startHash(s.length * seedString)
|
||||
var c = hiddenMagicA
|
||||
var k = hiddenMagicB
|
||||
|
|
@ -239,7 +239,7 @@ object MurmurHash {
|
|||
* where the order of appearance of elements does not matter.
|
||||
* This is useful for hashing sets, for example.
|
||||
*/
|
||||
def symmetricHash[T](xs: TraversableOnce[T], seed: Int) = {
|
||||
def symmetricHash[T](xs: TraversableOnce[T], seed: Int): Int = {
|
||||
var a, b, n = 0
|
||||
var c = 1
|
||||
xs.foreach(i ⇒ {
|
||||
|
|
|
|||
|
|
@ -5,8 +5,7 @@
|
|||
package akka.routing
|
||||
|
||||
import akka.actor.{ Actor, ActorRef }
|
||||
import java.util.concurrent.ConcurrentSkipListSet
|
||||
import scala.collection.JavaConversions._
|
||||
import java.util.{ Set, TreeSet }
|
||||
|
||||
sealed trait ListenerMessage
|
||||
case class Listen(listener: ActorRef) extends ListenerMessage
|
||||
|
|
@ -25,13 +24,29 @@ case class WithListeners(f: (ActorRef) ⇒ Unit) extends ListenerMessage
|
|||
* Send <code>WithListeners(fun)</code> to traverse the current listeners.
|
||||
*/
|
||||
trait Listeners { self: Actor ⇒
|
||||
protected val listeners = new ConcurrentSkipListSet[ActorRef]
|
||||
protected val listeners: Set[ActorRef] = new TreeSet[ActorRef]
|
||||
|
||||
/**
|
||||
* Chain this into the receive function.
|
||||
*
|
||||
* {{ def receive = listenerManagement orElse … }}
|
||||
*/
|
||||
protected def listenerManagement: Actor.Receive = {
|
||||
case Listen(l) ⇒ listeners add l
|
||||
case Deafen(l) ⇒ listeners remove l
|
||||
case WithListeners(f) ⇒ listeners foreach f
|
||||
case Listen(l) ⇒ listeners add l
|
||||
case Deafen(l) ⇒ listeners remove l
|
||||
case WithListeners(f) ⇒
|
||||
val i = listeners.iterator
|
||||
while (i.hasNext) f(i.next)
|
||||
}
|
||||
|
||||
protected def gossip(msg: Any) = listeners foreach (_ ! msg)
|
||||
/**
|
||||
* Sends the supplied message to all current listeners using the provided sender as sender.
|
||||
*
|
||||
* @param msg
|
||||
* @param sender
|
||||
*/
|
||||
protected def gossip(msg: Any)(implicit sender: ActorRef = null): Unit = {
|
||||
val i = listeners.iterator
|
||||
while (i.hasNext) i.next ! msg
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -6,14 +6,12 @@ package akka.routing
|
|||
import akka.actor._
|
||||
import akka.util.Duration
|
||||
import akka.util.duration._
|
||||
import akka.config.ConfigurationException
|
||||
import akka.ConfigurationException
|
||||
import akka.pattern.pipe
|
||||
import akka.pattern.AskSupport
|
||||
import com.typesafe.config.Config
|
||||
import scala.collection.JavaConversions.iterableAsScalaIterable
|
||||
import java.util.concurrent.atomic.{ AtomicLong, AtomicBoolean }
|
||||
import java.util.concurrent.TimeUnit
|
||||
import java.util.concurrent.locks.ReentrantLock
|
||||
import akka.jsr166y.ThreadLocalRandom
|
||||
import akka.util.Unsafe
|
||||
import akka.dispatch.Dispatchers
|
||||
|
|
@ -31,11 +29,17 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup
|
|||
_supervisor,
|
||||
_path) {
|
||||
|
||||
// verify that a BalancingDispatcher is not used with a Router
|
||||
if (_system.dispatchers.isBalancingDispatcher(_props.dispatcher) && _props.routerConfig != NoRouter)
|
||||
throw new ConfigurationException(
|
||||
"Configuration for actor [" + _path.toString +
|
||||
"] is invalid - you can not use a 'BalancingDispatcher' together with any type of 'Router'")
|
||||
|
||||
/*
|
||||
* CAUTION: RoutedActorRef is PROBLEMATIC
|
||||
* ======================================
|
||||
*
|
||||
* We are constructing/assembling the children outside of the scope of the
|
||||
*
|
||||
* We are constructing/assembling the children outside of the scope of the
|
||||
* Router actor, inserting them in its childrenRef list, which is not at all
|
||||
* synchronized. This is done exactly once at start-up, all other accesses
|
||||
* are done from the Router actor. This means that the only thing which is
|
||||
|
|
@ -49,12 +53,11 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup
|
|||
ref: InternalActorRef,
|
||||
props: Props,
|
||||
supervisor: InternalActorRef,
|
||||
receiveTimeout: Option[Duration]): ActorCell =
|
||||
{
|
||||
val cell = super.newActorCell(system, ref, props, supervisor, receiveTimeout)
|
||||
Unsafe.instance.monitorEnter(cell)
|
||||
cell
|
||||
}
|
||||
receiveTimeout: Option[Duration]): ActorCell = {
|
||||
val cell = super.newActorCell(system, ref, props, supervisor, receiveTimeout)
|
||||
Unsafe.instance.monitorEnter(cell)
|
||||
cell
|
||||
}
|
||||
|
||||
private[akka] val routerConfig = _props.routerConfig
|
||||
private[akka] val routeeProps = _props.copy(routerConfig = NoRouter)
|
||||
|
|
@ -171,7 +174,7 @@ trait RouterConfig {
|
|||
|
||||
def createRoute(routeeProps: Props, routeeProvider: RouteeProvider): Route
|
||||
|
||||
def createRouteeProvider(context: ActorContext) = new RouteeProvider(context, resizer)
|
||||
def createRouteeProvider(context: ActorContext): RouteeProvider = new RouteeProvider(context, resizer)
|
||||
|
||||
def createActor(): Router = new Router {
|
||||
override def supervisorStrategy: SupervisorStrategy = RouterConfig.this.supervisorStrategy
|
||||
|
|
@ -192,7 +195,8 @@ trait RouterConfig {
|
|||
*/
|
||||
def withFallback(other: RouterConfig): RouterConfig = this
|
||||
|
||||
protected def toAll(sender: ActorRef, routees: Iterable[ActorRef]): Iterable[Destination] = routees.map(Destination(sender, _))
|
||||
protected def toAll(sender: ActorRef, routees: Iterable[ActorRef]): Iterable[Destination] =
|
||||
routees.map(Destination(sender, _))
|
||||
|
||||
/**
|
||||
* Routers with dynamically resizable number of routees return the [[akka.routing.Resizer]]
|
||||
|
|
@ -215,9 +219,7 @@ class RouteeProvider(val context: ActorContext, val resizer: Option[Resizer]) {
|
|||
* Not thread safe, but intended to be called from protected points, such as
|
||||
* `RouterConfig.createRoute` and `Resizer.resize`.
|
||||
*/
|
||||
def registerRoutees(routees: IndexedSeq[ActorRef]): Unit = {
|
||||
routedRef.addRoutees(routees)
|
||||
}
|
||||
def registerRoutees(routees: IndexedSeq[ActorRef]): Unit = routedRef.addRoutees(routees)
|
||||
|
||||
/**
|
||||
* Adds the routees to the router.
|
||||
|
|
@ -237,9 +239,7 @@ class RouteeProvider(val context: ActorContext, val resizer: Option[Resizer]) {
|
|||
* Not thread safe, but intended to be called from protected points, such as
|
||||
* `Resizer.resize`.
|
||||
*/
|
||||
def unregisterRoutees(routees: IndexedSeq[ActorRef]): Unit = {
|
||||
routedRef.removeRoutees(routees)
|
||||
}
|
||||
def unregisterRoutees(routees: IndexedSeq[ActorRef]): Unit = routedRef.removeRoutees(routees)
|
||||
|
||||
def createRoutees(props: Props, nrOfInstances: Int, routees: Iterable[String]): IndexedSeq[ActorRef] =
|
||||
(nrOfInstances, routees) match {
|
||||
|
|
@ -250,11 +250,8 @@ class RouteeProvider(val context: ActorContext, val resizer: Option[Resizer]) {
|
|||
case (_, xs) ⇒ xs.map(context.actorFor(_))(scala.collection.breakOut)
|
||||
}
|
||||
|
||||
def createAndRegisterRoutees(props: Props, nrOfInstances: Int, routees: Iterable[String]): Unit = {
|
||||
if (resizer.isEmpty) {
|
||||
registerRoutees(createRoutees(props, nrOfInstances, routees))
|
||||
}
|
||||
}
|
||||
def createAndRegisterRoutees(props: Props, nrOfInstances: Int, routees: Iterable[String]): Unit =
|
||||
if (resizer.isEmpty) registerRoutees(createRoutees(props, nrOfInstances, routees))
|
||||
|
||||
/**
|
||||
* All routees of the router
|
||||
|
|
@ -262,7 +259,6 @@ class RouteeProvider(val context: ActorContext, val resizer: Option[Resizer]) {
|
|||
def routees: IndexedSeq[ActorRef] = routedRef.routees
|
||||
|
||||
private def routedRef = context.self.asInstanceOf[RoutedActorRef]
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -305,8 +301,8 @@ trait Router extends Actor {
|
|||
final def receive = ({
|
||||
|
||||
case Router.Resize ⇒
|
||||
try ref.routerConfig.resizer foreach (_.resize(ref.routeeProps, ref.routeeProvider))
|
||||
finally assert(ref.resizeInProgress.getAndSet(false))
|
||||
val ab = ref.resizeInProgress
|
||||
if (ab.get) try ref.routerConfig.resizer foreach (_.resize(ref.routeeProps, ref.routeeProvider)) finally ab.set(false)
|
||||
|
||||
case Terminated(child) ⇒
|
||||
ref.removeRoutees(IndexedSeq(child))
|
||||
|
|
@ -321,6 +317,9 @@ trait Router extends Actor {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private object Router {
|
||||
|
||||
case object Resize
|
||||
|
|
@ -374,9 +373,9 @@ case class Destination(sender: ActorRef, recipient: ActorRef)
|
|||
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
|
||||
abstract class NoRouter extends RouterConfig
|
||||
case object NoRouter extends NoRouter {
|
||||
def createRoute(props: Props, routeeProvider: RouteeProvider): Route = null
|
||||
def createRoute(props: Props, routeeProvider: RouteeProvider): Route = null // FIXME, null, really??
|
||||
def routerDispatcher: String = ""
|
||||
def supervisorStrategy = null
|
||||
def supervisorStrategy = null // FIXME null, really??
|
||||
override def withFallback(other: RouterConfig): RouterConfig = other
|
||||
|
||||
/**
|
||||
|
|
@ -406,9 +405,7 @@ case object FromConfig extends FromConfig {
|
|||
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
|
||||
class FromConfig(val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
||||
extends RouterConfig
|
||||
with Product
|
||||
with Serializable
|
||||
with Equals {
|
||||
with Serializable {
|
||||
|
||||
def this() = this(Dispatchers.DefaultDispatcherId)
|
||||
|
||||
|
|
@ -416,42 +413,14 @@ class FromConfig(val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
|||
throw new ConfigurationException("router " + routeeProvider.context.self + " needs external configuration from file (e.g. application.conf)")
|
||||
|
||||
def supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy
|
||||
|
||||
// open-coded case class to preserve binary compatibility, all deprecated for 2.1
|
||||
@deprecated("FromConfig does not make sense as case class", "2.0.1")
|
||||
override def productPrefix = "FromConfig"
|
||||
|
||||
@deprecated("FromConfig does not make sense as case class", "2.0.1")
|
||||
def productArity = 1
|
||||
|
||||
@deprecated("FromConfig does not make sense as case class", "2.0.1")
|
||||
def productElement(x: Int) = x match {
|
||||
case 0 ⇒ routerDispatcher
|
||||
case _ ⇒ throw new IndexOutOfBoundsException(x.toString)
|
||||
}
|
||||
|
||||
@deprecated("FromConfig does not make sense as case class", "2.0.1")
|
||||
def copy(d: String = Dispatchers.DefaultDispatcherId): FromConfig = new FromConfig(d)
|
||||
|
||||
@deprecated("FromConfig does not make sense as case class", "2.0.1")
|
||||
def canEqual(o: Any) = o.isInstanceOf[FromConfig]
|
||||
|
||||
@deprecated("FromConfig does not make sense as case class", "2.0.1")
|
||||
override def hashCode = ScalaRunTime._hashCode(this)
|
||||
|
||||
@deprecated("FromConfig does not make sense as case class", "2.0.1")
|
||||
override def toString = "FromConfig(" + routerDispatcher + ")"
|
||||
|
||||
@deprecated("FromConfig does not make sense as case class", "2.0.1")
|
||||
override def equals(other: Any): Boolean = other match {
|
||||
case FromConfig(x) ⇒ x == routerDispatcher
|
||||
case _ ⇒ false
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
object RoundRobinRouter {
|
||||
def apply(routees: Iterable[ActorRef]) = new RoundRobinRouter(routees = routees map (_.path.toString))
|
||||
/**
|
||||
* Creates a new RoundRobinRouter, routing to the specified routees
|
||||
*/
|
||||
def apply(routees: Iterable[ActorRef]): RoundRobinRouter =
|
||||
new RoundRobinRouter(routees = routees map (_.path.toString))
|
||||
|
||||
/**
|
||||
* Java API to create router with the supplied 'routees' actors.
|
||||
|
|
@ -512,9 +481,7 @@ case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] =
|
|||
* Constructor that sets nrOfInstances to be created.
|
||||
* Java API
|
||||
*/
|
||||
def this(nr: Int) = {
|
||||
this(nrOfInstances = nr)
|
||||
}
|
||||
def this(nr: Int) = this(nrOfInstances = nr)
|
||||
|
||||
/**
|
||||
* Constructor that sets the routees to be used.
|
||||
|
|
@ -522,9 +489,7 @@ case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] =
|
|||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
def this(routeePaths: java.lang.Iterable[String]) = {
|
||||
this(routees = iterableAsScalaIterable(routeePaths))
|
||||
}
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = iterableAsScalaIterable(routeePaths))
|
||||
|
||||
/**
|
||||
* Constructor that sets the resizer to be used.
|
||||
|
|
@ -535,13 +500,13 @@ case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] =
|
|||
/**
|
||||
* Java API for setting routerDispatcher
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId)
|
||||
def withDispatcher(dispatcherId: String): RoundRobinRouter = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
/**
|
||||
* Java API for setting the supervisor strategy to be used for the “head”
|
||||
* Router actor.
|
||||
*/
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy) = copy(supervisorStrategy = strategy)
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy): RoundRobinRouter = copy(supervisorStrategy = strategy)
|
||||
}
|
||||
|
||||
trait RoundRobinLike { this: RouterConfig ⇒
|
||||
|
|
@ -571,7 +536,10 @@ trait RoundRobinLike { this: RouterConfig ⇒
|
|||
}
|
||||
|
||||
object RandomRouter {
|
||||
def apply(routees: Iterable[ActorRef]) = new RandomRouter(routees = routees map (_.path.toString))
|
||||
/**
|
||||
* Creates a new RandomRouter, routing to the specified routees
|
||||
*/
|
||||
def apply(routees: Iterable[ActorRef]): RandomRouter = new RandomRouter(routees = routees map (_.path.toString))
|
||||
|
||||
/**
|
||||
* Java API to create router with the supplied 'routees' actors.
|
||||
|
|
@ -632,9 +600,7 @@ case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil,
|
|||
* Constructor that sets nrOfInstances to be created.
|
||||
* Java API
|
||||
*/
|
||||
def this(nr: Int) = {
|
||||
this(nrOfInstances = nr)
|
||||
}
|
||||
def this(nr: Int) = this(nrOfInstances = nr)
|
||||
|
||||
/**
|
||||
* Constructor that sets the routees to be used.
|
||||
|
|
@ -642,9 +608,7 @@ case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil,
|
|||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
def this(routeePaths: java.lang.Iterable[String]) = {
|
||||
this(routees = iterableAsScalaIterable(routeePaths))
|
||||
}
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = iterableAsScalaIterable(routeePaths))
|
||||
|
||||
/**
|
||||
* Constructor that sets the resizer to be used.
|
||||
|
|
@ -655,13 +619,13 @@ case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil,
|
|||
/**
|
||||
* Java API for setting routerDispatcher
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId)
|
||||
def withDispatcher(dispatcherId: String): RandomRouter = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
/**
|
||||
* Java API for setting the supervisor strategy to be used for the “head”
|
||||
* Router actor.
|
||||
*/
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy) = copy(supervisorStrategy = strategy)
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy): RandomRouter = copy(supervisorStrategy = strategy)
|
||||
}
|
||||
|
||||
trait RandomLike { this: RouterConfig ⇒
|
||||
|
|
@ -688,7 +652,11 @@ trait RandomLike { this: RouterConfig ⇒
|
|||
}
|
||||
|
||||
object SmallestMailboxRouter {
|
||||
def apply(routees: Iterable[ActorRef]) = new SmallestMailboxRouter(routees = routees map (_.path.toString))
|
||||
/**
|
||||
* Creates a new SmallestMailboxRouter, routing to the specified routees
|
||||
*/
|
||||
def apply(routees: Iterable[ActorRef]): SmallestMailboxRouter =
|
||||
new SmallestMailboxRouter(routees = routees map (_.path.toString))
|
||||
|
||||
/**
|
||||
* Java API to create router with the supplied 'routees' actors.
|
||||
|
|
@ -758,9 +726,7 @@ case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[Strin
|
|||
* Constructor that sets nrOfInstances to be created.
|
||||
* Java API
|
||||
*/
|
||||
def this(nr: Int) = {
|
||||
this(nrOfInstances = nr)
|
||||
}
|
||||
def this(nr: Int) = this(nrOfInstances = nr)
|
||||
|
||||
/**
|
||||
* Constructor that sets the routees to be used.
|
||||
|
|
@ -768,9 +734,7 @@ case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[Strin
|
|||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
def this(routeePaths: java.lang.Iterable[String]) = {
|
||||
this(routees = iterableAsScalaIterable(routeePaths))
|
||||
}
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = iterableAsScalaIterable(routeePaths))
|
||||
|
||||
/**
|
||||
* Constructor that sets the resizer to be used.
|
||||
|
|
@ -781,19 +745,16 @@ case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[Strin
|
|||
/**
|
||||
* Java API for setting routerDispatcher
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId)
|
||||
def withDispatcher(dispatcherId: String): SmallestMailboxRouter = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
/**
|
||||
* Java API for setting the supervisor strategy to be used for the “head”
|
||||
* Router actor.
|
||||
*/
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy) = copy(supervisorStrategy = strategy)
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy): SmallestMailboxRouter = copy(supervisorStrategy = strategy)
|
||||
}
|
||||
|
||||
trait SmallestMailboxLike { this: RouterConfig ⇒
|
||||
|
||||
import java.security.SecureRandom
|
||||
|
||||
def nrOfInstances: Int
|
||||
|
||||
def routees: Iterable[String]
|
||||
|
|
@ -895,7 +856,10 @@ trait SmallestMailboxLike { this: RouterConfig ⇒
|
|||
}
|
||||
|
||||
object BroadcastRouter {
|
||||
def apply(routees: Iterable[ActorRef]) = new BroadcastRouter(routees = routees map (_.path.toString))
|
||||
/**
|
||||
* Creates a new BroadcastRouter, routing to the specified routees
|
||||
*/
|
||||
def apply(routees: Iterable[ActorRef]): BroadcastRouter = new BroadcastRouter(routees = routees map (_.path.toString))
|
||||
|
||||
/**
|
||||
* Java API to create router with the supplied 'routees' actors.
|
||||
|
|
@ -956,9 +920,7 @@ case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = N
|
|||
* Constructor that sets nrOfInstances to be created.
|
||||
* Java API
|
||||
*/
|
||||
def this(nr: Int) = {
|
||||
this(nrOfInstances = nr)
|
||||
}
|
||||
def this(nr: Int) = this(nrOfInstances = nr)
|
||||
|
||||
/**
|
||||
* Constructor that sets the routees to be used.
|
||||
|
|
@ -966,9 +928,7 @@ case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = N
|
|||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
def this(routeePaths: java.lang.Iterable[String]) = {
|
||||
this(routees = iterableAsScalaIterable(routeePaths))
|
||||
}
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = iterableAsScalaIterable(routeePaths))
|
||||
|
||||
/**
|
||||
* Constructor that sets the resizer to be used.
|
||||
|
|
@ -979,13 +939,13 @@ case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = N
|
|||
/**
|
||||
* Java API for setting routerDispatcher
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId)
|
||||
def withDispatcher(dispatcherId: String): BroadcastRouter = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
/**
|
||||
* Java API for setting the supervisor strategy to be used for the “head”
|
||||
* Router actor.
|
||||
*/
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy) = copy(supervisorStrategy = strategy)
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy): BroadcastRouter = copy(supervisorStrategy = strategy)
|
||||
}
|
||||
|
||||
trait BroadcastLike { this: RouterConfig ⇒
|
||||
|
|
@ -1004,7 +964,11 @@ trait BroadcastLike { this: RouterConfig ⇒
|
|||
}
|
||||
|
||||
object ScatterGatherFirstCompletedRouter {
|
||||
def apply(routees: Iterable[ActorRef], within: Duration) = new ScatterGatherFirstCompletedRouter(routees = routees map (_.path.toString), within = within)
|
||||
/**
|
||||
* Creates a new ScatterGatherFirstCompletedRouter, routing to the specified routees, timing out after the specified Duration
|
||||
*/
|
||||
def apply(routees: Iterable[ActorRef], within: Duration): ScatterGatherFirstCompletedRouter =
|
||||
new ScatterGatherFirstCompletedRouter(routees = routees map (_.path.toString), within = within)
|
||||
|
||||
/**
|
||||
* Java API to create router with the supplied 'routees' actors.
|
||||
|
|
@ -1071,9 +1035,7 @@ case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: It
|
|||
* Constructor that sets nrOfInstances to be created.
|
||||
* Java API
|
||||
*/
|
||||
def this(nr: Int, w: Duration) = {
|
||||
this(nrOfInstances = nr, within = w)
|
||||
}
|
||||
def this(nr: Int, w: Duration) = this(nrOfInstances = nr, within = w)
|
||||
|
||||
/**
|
||||
* Constructor that sets the routees to be used.
|
||||
|
|
@ -1081,9 +1043,8 @@ case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: It
|
|||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
def this(routeePaths: java.lang.Iterable[String], w: Duration) = {
|
||||
def this(routeePaths: java.lang.Iterable[String], w: Duration) =
|
||||
this(routees = iterableAsScalaIterable(routeePaths), within = w)
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor that sets the resizer to be used.
|
||||
|
|
@ -1152,10 +1113,14 @@ trait Resizer {
|
|||
* This method is invoked only in the context of the Router actor in order to safely
|
||||
* create/stop children.
|
||||
*/
|
||||
def resize(props: Props, routeeProvider: RouteeProvider)
|
||||
def resize(props: Props, routeeProvider: RouteeProvider): Unit
|
||||
}
|
||||
|
||||
case object DefaultResizer {
|
||||
|
||||
/**
|
||||
* Creates a new DefaultResizer from the given configuration
|
||||
*/
|
||||
def apply(resizerConfig: Config): DefaultResizer =
|
||||
DefaultResizer(
|
||||
lowerBound = resizerConfig.getInt("lower-bound"),
|
||||
|
|
@ -1168,6 +1133,7 @@ case object DefaultResizer {
|
|||
messagesPerResize = resizerConfig.getInt("messages-per-resize"))
|
||||
}
|
||||
|
||||
//FIXME DOCUMENT ME
|
||||
case class DefaultResizer(
|
||||
/**
|
||||
* The fewest number of routees the router should ever have.
|
||||
|
|
@ -1242,7 +1208,7 @@ case class DefaultResizer(
|
|||
|
||||
def isTimeForResize(messageCounter: Long): Boolean = (messageCounter % messagesPerResize == 0)
|
||||
|
||||
def resize(props: Props, routeeProvider: RouteeProvider) {
|
||||
def resize(props: Props, routeeProvider: RouteeProvider): Unit = {
|
||||
val currentRoutees = routeeProvider.routees
|
||||
val requestedCapacity = capacity(currentRoutees)
|
||||
|
||||
|
|
@ -1260,7 +1226,7 @@ case class DefaultResizer(
|
|||
* Give concurrent messages a chance to be placed in mailbox before
|
||||
* sending PoisonPill.
|
||||
*/
|
||||
protected def delayedStop(scheduler: Scheduler, abandon: IndexedSeq[ActorRef]) {
|
||||
protected def delayedStop(scheduler: Scheduler, abandon: IndexedSeq[ActorRef]): Unit = {
|
||||
if (abandon.nonEmpty) {
|
||||
if (stopDelay <= Duration.Zero) {
|
||||
abandon foreach (_ ! PoisonPill)
|
||||
|
|
@ -1329,9 +1295,7 @@ case class DefaultResizer(
|
|||
* @param capacity current number of routees
|
||||
* @return proposed change in the capacity
|
||||
*/
|
||||
def filter(pressure: Int, capacity: Int): Int = {
|
||||
rampup(pressure, capacity) + backoff(pressure, capacity)
|
||||
}
|
||||
def filter(pressure: Int, capacity: Int): Int = rampup(pressure, capacity) + backoff(pressure, capacity)
|
||||
|
||||
/**
|
||||
* Computes a proposed positive (or zero) capacity delta using
|
||||
|
|
|
|||
|
|
@ -14,8 +14,6 @@ import akka.util.NonFatal
|
|||
import scala.collection.mutable.ArrayBuffer
|
||||
import java.io.NotSerializableException
|
||||
|
||||
case class NoSerializerFoundException(m: String) extends AkkaException(m)
|
||||
|
||||
object Serialization {
|
||||
|
||||
/**
|
||||
|
|
@ -85,7 +83,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
|
|||
/**
|
||||
* Returns the Serializer configured for the given object, returns the NullSerializer if it's null.
|
||||
*
|
||||
* @throws akka.config.ConfigurationException if no `serialization-bindings` is configured for the
|
||||
* @throws akka.ConfigurationException if no `serialization-bindings` is configured for the
|
||||
* class of the object
|
||||
*/
|
||||
def findSerializerFor(o: AnyRef): Serializer = o match {
|
||||
|
|
@ -120,9 +118,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
|
|||
possibilities(0)._2
|
||||
}
|
||||
serializerMap.putIfAbsent(clazz, ser) match {
|
||||
case null ⇒
|
||||
log.debug("Using serializer[{}] for message [{}]", ser.getClass.getName, clazz.getName)
|
||||
ser
|
||||
case null ⇒ log.debug("Using serializer[{}] for message [{}]", ser.getClass.getName, clazz.getName); ser
|
||||
case some ⇒ some
|
||||
}
|
||||
case ser ⇒ ser
|
||||
|
|
@ -140,10 +136,8 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
|
|||
* A Map of serializer from alias to implementation (class implementing akka.serialization.Serializer)
|
||||
* By default always contains the following mapping: "java" -> akka.serialization.JavaSerializer
|
||||
*/
|
||||
private val serializers: Map[String, Serializer] = {
|
||||
for ((k: String, v: String) ← settings.Serializers)
|
||||
yield k -> serializerOf(v).fold(throw _, identity)
|
||||
}
|
||||
private val serializers: Map[String, Serializer] =
|
||||
for ((k: String, v: String) ← settings.Serializers) yield k -> serializerOf(v).fold(throw _, identity)
|
||||
|
||||
/**
|
||||
* bindings is a Seq of tuple representing the mapping from Class to Serializer.
|
||||
|
|
|
|||
|
|
@ -6,7 +6,6 @@ package akka.serialization
|
|||
|
||||
import java.io.{ ObjectOutputStream, ByteArrayOutputStream, ObjectInputStream, ByteArrayInputStream }
|
||||
import akka.util.ClassLoaderObjectInputStream
|
||||
import akka.actor.DynamicAccess
|
||||
import akka.actor.ExtendedActorSystem
|
||||
import scala.util.DynamicVariable
|
||||
|
||||
|
|
|
|||
|
|
@ -8,6 +8,12 @@ import java.util.concurrent.locks.ReentrantLock
|
|||
import java.util.concurrent.{ TimeUnit, BlockingQueue }
|
||||
import java.util.{ AbstractQueue, Queue, Collection, Iterator }
|
||||
|
||||
/**
|
||||
* BoundedBlockingQueue wraps any Queue and turns the result into a BlockingQueue with a limited capacity
|
||||
* @param maxCapacity - the maximum capacity of this Queue, needs to be > 0
|
||||
* @param backing - the backing Queue
|
||||
* @tparam E - The type of the contents of this Queue
|
||||
*/
|
||||
class BoundedBlockingQueue[E <: AnyRef](
|
||||
val maxCapacity: Int, private val backing: Queue[E]) extends AbstractQueue[E] with BlockingQueue[E] {
|
||||
|
||||
|
|
@ -22,7 +28,7 @@ class BoundedBlockingQueue[E <: AnyRef](
|
|||
require(maxCapacity > 0)
|
||||
}
|
||||
|
||||
protected val lock = new ReentrantLock(false)
|
||||
protected val lock = new ReentrantLock(false) // TODO might want to switch to ReentrantReadWriteLock
|
||||
|
||||
private val notEmpty = lock.newCondition()
|
||||
private val notFull = lock.newCondition()
|
||||
|
|
|
|||
|
|
@ -11,6 +11,7 @@ import scala.collection.mutable.{ Builder, WrappedArray }
|
|||
import scala.collection.immutable.{ IndexedSeq, VectorBuilder }
|
||||
import scala.collection.generic.CanBuildFrom
|
||||
|
||||
//FIXME MORE DOCS
|
||||
object ByteString {
|
||||
|
||||
/**
|
||||
|
|
@ -53,15 +54,16 @@ object ByteString {
|
|||
|
||||
val empty: ByteString = CompactByteString(Array.empty[Byte])
|
||||
|
||||
def newBuilder = new ByteStringBuilder
|
||||
def newBuilder: ByteStringBuilder = new ByteStringBuilder
|
||||
|
||||
implicit def canBuildFrom = new CanBuildFrom[TraversableOnce[Byte], Byte, ByteString] {
|
||||
def apply(from: TraversableOnce[Byte]) = newBuilder
|
||||
def apply() = newBuilder
|
||||
}
|
||||
implicit val canBuildFrom: CanBuildFrom[TraversableOnce[Byte], Byte, ByteString] =
|
||||
new CanBuildFrom[TraversableOnce[Byte], Byte, ByteString] {
|
||||
def apply(ignore: TraversableOnce[Byte]): ByteStringBuilder = newBuilder
|
||||
def apply(): ByteStringBuilder = newBuilder
|
||||
}
|
||||
|
||||
private[akka] object ByteString1C {
|
||||
def apply(bytes: Array[Byte]) = new ByteString1C(bytes)
|
||||
def apply(bytes: Array[Byte]): ByteString1C = new ByteString1C(bytes)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -71,7 +73,7 @@ object ByteString {
|
|||
final class ByteString1C private (private val bytes: Array[Byte]) extends CompactByteString {
|
||||
def apply(idx: Int): Byte = bytes(idx)
|
||||
|
||||
override def length = bytes.length
|
||||
override def length: Int = bytes.length
|
||||
|
||||
def toArray: Array[Byte] = bytes.clone
|
||||
|
||||
|
|
@ -81,13 +83,11 @@ object ByteString {
|
|||
|
||||
def compact: ByteString1C = this
|
||||
|
||||
def asByteBuffer: ByteBuffer =
|
||||
toByteString1.asByteBuffer
|
||||
def asByteBuffer: ByteBuffer = toByteString1.asByteBuffer
|
||||
|
||||
def decodeString(charset: String): String = new String(bytes, charset)
|
||||
|
||||
def ++(that: ByteString): ByteString =
|
||||
if (!that.isEmpty) toByteString1 ++ that else this
|
||||
def ++(that: ByteString): ByteString = if (!that.isEmpty) toByteString1 ++ that else this
|
||||
|
||||
override def slice(from: Int, until: Int): ByteString =
|
||||
if ((from != 0) || (until != length)) toByteString1.slice(from, until)
|
||||
|
|
@ -96,12 +96,11 @@ object ByteString {
|
|||
override def copyToArray[A >: Byte](xs: Array[A], start: Int, len: Int): Unit =
|
||||
toByteString1.copyToArray(xs, start, len)
|
||||
|
||||
def copyToBuffer(buffer: ByteBuffer): Int =
|
||||
toByteString1.copyToBuffer(buffer)
|
||||
def copyToBuffer(buffer: ByteBuffer): Int = toByteString1.copyToBuffer(buffer)
|
||||
}
|
||||
|
||||
private[akka] object ByteString1 {
|
||||
def apply(bytes: Array[Byte]) = new ByteString1(bytes)
|
||||
def apply(bytes: Array[Byte]): ByteString1 = new ByteString1(bytes)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -113,7 +112,7 @@ object ByteString {
|
|||
|
||||
def apply(idx: Int): Byte = bytes(checkRangeConvert(idx))
|
||||
|
||||
private def checkRangeConvert(index: Int) = {
|
||||
private def checkRangeConvert(index: Int): Int = {
|
||||
if (0 <= index && length > index)
|
||||
index + startIndex
|
||||
else
|
||||
|
|
@ -128,8 +127,7 @@ object ByteString {
|
|||
|
||||
override def clone: CompactByteString = ByteString1C(toArray)
|
||||
|
||||
def compact: CompactByteString =
|
||||
if (length == bytes.length) ByteString1C(bytes) else clone
|
||||
def compact: CompactByteString = if (length == bytes.length) ByteString1C(bytes) else clone
|
||||
|
||||
def asByteBuffer: ByteBuffer = {
|
||||
val buffer = ByteBuffer.wrap(bytes, startIndex, length).asReadOnlyBuffer
|
||||
|
|
@ -161,7 +159,6 @@ object ByteString {
|
|||
if (copyLength > 0) buffer.put(bytes, startIndex, copyLength)
|
||||
copyLength
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private[akka] object ByteStrings {
|
||||
|
|
@ -198,10 +195,11 @@ object ByteString {
|
|||
}
|
||||
|
||||
// 0: both empty, 1: 2nd empty, 2: 1st empty, 3: neither empty
|
||||
// Using length to check emptiness is prohibited by law
|
||||
def compare(b1: ByteString, b2: ByteString): Int =
|
||||
if (b1.length == 0)
|
||||
if (b2.length == 0) 0 else 2
|
||||
else if (b2.length == 0) 1 else 3
|
||||
if (b1.isEmpty)
|
||||
if (b2.isEmpty) 0 else 2
|
||||
else if (b2.isEmpty) 1 else 3
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -439,7 +437,7 @@ final class ByteStringBuilder extends Builder[Byte, ByteString] {
|
|||
private var _tempLength = 0
|
||||
private var _tempCapacity = 0
|
||||
|
||||
private def clearTemp() {
|
||||
private def clearTemp(): Unit = {
|
||||
if (_tempLength > 0) {
|
||||
val arr = new Array[Byte](_tempLength)
|
||||
Array.copy(_temp, 0, arr, 0, _tempLength)
|
||||
|
|
@ -448,14 +446,14 @@ final class ByteStringBuilder extends Builder[Byte, ByteString] {
|
|||
}
|
||||
}
|
||||
|
||||
private def resizeTemp(size: Int) {
|
||||
private def resizeTemp(size: Int): Unit = {
|
||||
val newtemp = new Array[Byte](size)
|
||||
if (_tempLength > 0) Array.copy(_temp, 0, newtemp, 0, _tempLength)
|
||||
_temp = newtemp
|
||||
_tempCapacity = _temp.length
|
||||
}
|
||||
|
||||
private def ensureTempSize(size: Int) {
|
||||
private def ensureTempSize(size: Int): Unit = {
|
||||
if (_tempCapacity < size || _tempCapacity == 0) {
|
||||
var newSize = if (_tempCapacity == 0) 16 else _tempCapacity * 2
|
||||
while (newSize < size) newSize *= 2
|
||||
|
|
@ -498,7 +496,7 @@ final class ByteStringBuilder extends Builder[Byte, ByteString] {
|
|||
this
|
||||
}
|
||||
|
||||
def clear() {
|
||||
def clear(): Unit = {
|
||||
_builder.clear
|
||||
_length = 0
|
||||
_tempLength = 0
|
||||
|
|
|
|||
|
|
@ -6,6 +6,13 @@ package akka.util
|
|||
|
||||
import java.io.{ InputStream, ObjectInputStream, ObjectStreamClass }
|
||||
|
||||
/**
|
||||
* ClassLoaderObjectInputStream tries to utilize the provided ClassLoader to load Classes and falls
|
||||
* back to ObjectInputStreams resolver.
|
||||
*
|
||||
* @param classLoader - the ClassLoader which is to be used primarily
|
||||
* @param is - the InputStream that is wrapped
|
||||
*/
|
||||
class ClassLoaderObjectInputStream(classLoader: ClassLoader, is: InputStream) extends ObjectInputStream(is) {
|
||||
override protected def resolveClass(objectStreamClass: ObjectStreamClass): Class[_] =
|
||||
try Class.forName(objectStreamClass.getName, false, classLoader) catch {
|
||||
|
|
|
|||
|
|
@ -3,7 +3,7 @@
|
|||
*/
|
||||
|
||||
package akka.util
|
||||
|
||||
//FIXME DOCS!
|
||||
object Convert {
|
||||
|
||||
def intToBytes(value: Int): Array[Byte] = {
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@
|
|||
package akka.util
|
||||
|
||||
import java.security.{ MessageDigest, SecureRandom }
|
||||
|
||||
//FIXME DOCS
|
||||
object Crypt {
|
||||
val hex = "0123456789ABCDEF"
|
||||
val lineSeparator = System.getProperty("line.separator")
|
||||
|
|
@ -32,7 +32,7 @@ object Crypt {
|
|||
}
|
||||
|
||||
def hexify(bytes: Array[Byte]): String = {
|
||||
val builder = new StringBuilder
|
||||
val builder = new StringBuilder(bytes.length * 2)
|
||||
bytes.foreach { byte ⇒ builder.append(hex.charAt((byte & 0xF0) >> 4)).append(hex.charAt(byte & 0xF)) }
|
||||
builder.toString
|
||||
}
|
||||
|
|
|
|||
|
|
@ -110,6 +110,7 @@ object Duration {
|
|||
}
|
||||
|
||||
val Zero: FiniteDuration = new FiniteDuration(0, NANOSECONDS)
|
||||
|
||||
val Undefined: Duration = new Duration with Infinite {
|
||||
override def toString = "Duration.Undefined"
|
||||
override def equals(other: Any) = other.asInstanceOf[AnyRef] eq this
|
||||
|
|
@ -166,8 +167,8 @@ object Duration {
|
|||
* including itself.
|
||||
*/
|
||||
val Inf: Duration = new Duration with Infinite {
|
||||
override def toString = "Duration.Inf"
|
||||
def compare(other: Duration) = if (other eq this) 0 else 1
|
||||
override def toString: String = "Duration.Inf"
|
||||
def compare(other: Duration): Int = if (other eq this) 0 else 1
|
||||
def unary_- : Duration = MinusInf
|
||||
}
|
||||
|
||||
|
|
@ -177,7 +178,7 @@ object Duration {
|
|||
*/
|
||||
val MinusInf: Duration = new Duration with Infinite {
|
||||
override def toString = "Duration.MinusInf"
|
||||
def compare(other: Duration) = if (other eq this) 0 else -1
|
||||
def compare(other: Duration): Int = if (other eq this) 0 else -1
|
||||
def unary_- : Duration = Inf
|
||||
}
|
||||
|
||||
|
|
@ -188,7 +189,7 @@ object Duration {
|
|||
def parse(s: String): Duration = unapply(s).get
|
||||
|
||||
implicit object DurationIsOrdered extends Ordering[Duration] {
|
||||
def compare(a: Duration, b: Duration) = a compare b
|
||||
def compare(a: Duration, b: Duration): Int = a compare b
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -263,17 +264,17 @@ abstract class Duration extends Serializable with Ordered[Duration] {
|
|||
def fromNow: Deadline = Deadline.now + this
|
||||
|
||||
// Java API
|
||||
def lt(other: Duration) = this < other
|
||||
def lteq(other: Duration) = this <= other
|
||||
def gt(other: Duration) = this > other
|
||||
def gteq(other: Duration) = this >= other
|
||||
def plus(other: Duration) = this + other
|
||||
def minus(other: Duration) = this - other
|
||||
def mul(factor: Double) = this * factor
|
||||
def div(factor: Double) = this / factor
|
||||
def div(other: Duration) = this / other
|
||||
def neg() = -this
|
||||
def isFinite() = finite_?
|
||||
def lt(other: Duration): Boolean = this < other
|
||||
def lteq(other: Duration): Boolean = this <= other
|
||||
def gt(other: Duration): Boolean = this > other
|
||||
def gteq(other: Duration): Boolean = this >= other
|
||||
def plus(other: Duration): Duration = this + other
|
||||
def minus(other: Duration): Duration = this - other
|
||||
def mul(factor: Double): Duration = this * factor
|
||||
def div(factor: Double): Duration = this / factor
|
||||
def div(other: Duration): Double = this / other
|
||||
def neg(): Duration = -this
|
||||
def isFinite(): Boolean = finite_?
|
||||
}
|
||||
|
||||
object FiniteDuration {
|
||||
|
|
@ -349,31 +350,19 @@ class FiniteDuration(val length: Long, val unit: TimeUnit) extends Duration {
|
|||
else c
|
||||
}
|
||||
|
||||
def +(other: Duration) = {
|
||||
if (!other.finite_?) {
|
||||
other
|
||||
} else {
|
||||
fromNanos(add(toNanos, other.toNanos))
|
||||
}
|
||||
}
|
||||
def +(other: Duration): Duration = if (!other.finite_?) other else fromNanos(add(toNanos, other.toNanos))
|
||||
|
||||
def -(other: Duration) = {
|
||||
if (!other.finite_?) {
|
||||
other
|
||||
} else {
|
||||
fromNanos(add(toNanos, -other.toNanos))
|
||||
}
|
||||
}
|
||||
def -(other: Duration): Duration = if (!other.finite_?) other else fromNanos(add(toNanos, -other.toNanos))
|
||||
|
||||
def *(factor: Double) = fromNanos(long2double(toNanos) * factor)
|
||||
def *(factor: Double): FiniteDuration = fromNanos(long2double(toNanos) * factor)
|
||||
|
||||
def /(factor: Double) = fromNanos(long2double(toNanos) / factor)
|
||||
def /(factor: Double): FiniteDuration = fromNanos(long2double(toNanos) / factor)
|
||||
|
||||
def /(other: Duration) = if (other.finite_?) long2double(toNanos) / other.toNanos else 0
|
||||
def /(other: Duration): Double = if (other.finite_?) long2double(toNanos) / other.toNanos else 0
|
||||
|
||||
def unary_- = Duration(-length, unit)
|
||||
def unary_- : FiniteDuration = Duration(-length, unit)
|
||||
|
||||
def finite_? = true
|
||||
def finite_? : Boolean = true
|
||||
|
||||
override def equals(other: Any) =
|
||||
(other.asInstanceOf[AnyRef] eq this) || other.isInstanceOf[FiniteDuration] &&
|
||||
|
|
@ -385,178 +374,74 @@ class FiniteDuration(val length: Long, val unit: TimeUnit) extends Duration {
|
|||
}
|
||||
}
|
||||
|
||||
class DurationInt(n: Int) {
|
||||
private[akka] trait DurationOps {
|
||||
import duration.Classifier
|
||||
protected def from(timeUnit: TimeUnit): FiniteDuration
|
||||
def nanoseconds: FiniteDuration = from(NANOSECONDS)
|
||||
def nanos: FiniteDuration = from(NANOSECONDS)
|
||||
def nanosecond: FiniteDuration = from(NANOSECONDS)
|
||||
def nano: FiniteDuration = from(NANOSECONDS)
|
||||
|
||||
def nanoseconds = Duration(n, NANOSECONDS)
|
||||
def nanos = Duration(n, NANOSECONDS)
|
||||
def nanosecond = Duration(n, NANOSECONDS)
|
||||
def nano = Duration(n, NANOSECONDS)
|
||||
def microseconds: FiniteDuration = from(MICROSECONDS)
|
||||
def micros: FiniteDuration = from(MICROSECONDS)
|
||||
def microsecond: FiniteDuration = from(MICROSECONDS)
|
||||
def micro: FiniteDuration = from(MICROSECONDS)
|
||||
|
||||
def microseconds = Duration(n, MICROSECONDS)
|
||||
def micros = Duration(n, MICROSECONDS)
|
||||
def microsecond = Duration(n, MICROSECONDS)
|
||||
def micro = Duration(n, MICROSECONDS)
|
||||
def milliseconds: FiniteDuration = from(MILLISECONDS)
|
||||
def millis: FiniteDuration = from(MILLISECONDS)
|
||||
def millisecond: FiniteDuration = from(MILLISECONDS)
|
||||
def milli: FiniteDuration = from(MILLISECONDS)
|
||||
|
||||
def milliseconds = Duration(n, MILLISECONDS)
|
||||
def millis = Duration(n, MILLISECONDS)
|
||||
def millisecond = Duration(n, MILLISECONDS)
|
||||
def milli = Duration(n, MILLISECONDS)
|
||||
def seconds: FiniteDuration = from(SECONDS)
|
||||
def second: FiniteDuration = from(SECONDS)
|
||||
|
||||
def seconds = Duration(n, SECONDS)
|
||||
def second = Duration(n, SECONDS)
|
||||
def minutes: FiniteDuration = from(MINUTES)
|
||||
def minute: FiniteDuration = from(MINUTES)
|
||||
|
||||
def minutes = Duration(n, MINUTES)
|
||||
def minute = Duration(n, MINUTES)
|
||||
def hours: FiniteDuration = from(HOURS)
|
||||
def hour: FiniteDuration = from(HOURS)
|
||||
|
||||
def hours = Duration(n, HOURS)
|
||||
def hour = Duration(n, HOURS)
|
||||
def days: FiniteDuration = from(DAYS)
|
||||
def day: FiniteDuration = from(DAYS)
|
||||
|
||||
def days = Duration(n, DAYS)
|
||||
def day = Duration(n, DAYS)
|
||||
def nanoseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(NANOSECONDS))
|
||||
def nanos[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(NANOSECONDS))
|
||||
def nanosecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(NANOSECONDS))
|
||||
def nano[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(NANOSECONDS))
|
||||
|
||||
def nanoseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS))
|
||||
def nanos[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS))
|
||||
def nanosecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS))
|
||||
def nano[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS))
|
||||
def microseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MICROSECONDS))
|
||||
def micros[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MICROSECONDS))
|
||||
def microsecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MICROSECONDS))
|
||||
def micro[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MICROSECONDS))
|
||||
|
||||
def microseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS))
|
||||
def micros[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS))
|
||||
def microsecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS))
|
||||
def micro[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS))
|
||||
def milliseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MILLISECONDS))
|
||||
def millis[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MILLISECONDS))
|
||||
def millisecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MILLISECONDS))
|
||||
def milli[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MILLISECONDS))
|
||||
|
||||
def milliseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS))
|
||||
def millis[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS))
|
||||
def millisecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS))
|
||||
def milli[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS))
|
||||
def seconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(SECONDS))
|
||||
def second[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(SECONDS))
|
||||
|
||||
def seconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, SECONDS))
|
||||
def second[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, SECONDS))
|
||||
def minutes[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MINUTES))
|
||||
def minute[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MINUTES))
|
||||
|
||||
def minutes[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MINUTES))
|
||||
def minute[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MINUTES))
|
||||
def hours[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(HOURS))
|
||||
def hour[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(HOURS))
|
||||
|
||||
def hours[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, HOURS))
|
||||
def hour[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, HOURS))
|
||||
|
||||
def days[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, DAYS))
|
||||
def day[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, DAYS))
|
||||
def days[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(DAYS))
|
||||
def day[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(DAYS))
|
||||
}
|
||||
|
||||
class DurationLong(n: Long) {
|
||||
import duration.Classifier
|
||||
|
||||
def nanoseconds = Duration(n, NANOSECONDS)
|
||||
def nanos = Duration(n, NANOSECONDS)
|
||||
def nanosecond = Duration(n, NANOSECONDS)
|
||||
def nano = Duration(n, NANOSECONDS)
|
||||
|
||||
def microseconds = Duration(n, MICROSECONDS)
|
||||
def micros = Duration(n, MICROSECONDS)
|
||||
def microsecond = Duration(n, MICROSECONDS)
|
||||
def micro = Duration(n, MICROSECONDS)
|
||||
|
||||
def milliseconds = Duration(n, MILLISECONDS)
|
||||
def millis = Duration(n, MILLISECONDS)
|
||||
def millisecond = Duration(n, MILLISECONDS)
|
||||
def milli = Duration(n, MILLISECONDS)
|
||||
|
||||
def seconds = Duration(n, SECONDS)
|
||||
def second = Duration(n, SECONDS)
|
||||
|
||||
def minutes = Duration(n, MINUTES)
|
||||
def minute = Duration(n, MINUTES)
|
||||
|
||||
def hours = Duration(n, HOURS)
|
||||
def hour = Duration(n, HOURS)
|
||||
|
||||
def days = Duration(n, DAYS)
|
||||
def day = Duration(n, DAYS)
|
||||
|
||||
def nanoseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS))
|
||||
def nanos[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS))
|
||||
def nanosecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS))
|
||||
def nano[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS))
|
||||
|
||||
def microseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS))
|
||||
def micros[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS))
|
||||
def microsecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS))
|
||||
def micro[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS))
|
||||
|
||||
def milliseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS))
|
||||
def millis[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS))
|
||||
def millisecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS))
|
||||
def milli[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS))
|
||||
|
||||
def seconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, SECONDS))
|
||||
def second[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, SECONDS))
|
||||
|
||||
def minutes[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MINUTES))
|
||||
def minute[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MINUTES))
|
||||
|
||||
def hours[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, HOURS))
|
||||
def hour[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, HOURS))
|
||||
|
||||
def days[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, DAYS))
|
||||
def day[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, DAYS))
|
||||
class DurationInt(n: Int) extends DurationOps {
|
||||
override protected def from(timeUnit: TimeUnit): FiniteDuration = Duration(n, timeUnit)
|
||||
}
|
||||
|
||||
class DurationDouble(d: Double) {
|
||||
import duration.Classifier
|
||||
class DurationLong(n: Long) extends DurationOps {
|
||||
override protected def from(timeUnit: TimeUnit): FiniteDuration = Duration(n, timeUnit)
|
||||
}
|
||||
|
||||
def nanoseconds = Duration(d, NANOSECONDS)
|
||||
def nanos = Duration(d, NANOSECONDS)
|
||||
def nanosecond = Duration(d, NANOSECONDS)
|
||||
def nano = Duration(d, NANOSECONDS)
|
||||
|
||||
def microseconds = Duration(d, MICROSECONDS)
|
||||
def micros = Duration(d, MICROSECONDS)
|
||||
def microsecond = Duration(d, MICROSECONDS)
|
||||
def micro = Duration(d, MICROSECONDS)
|
||||
|
||||
def milliseconds = Duration(d, MILLISECONDS)
|
||||
def millis = Duration(d, MILLISECONDS)
|
||||
def millisecond = Duration(d, MILLISECONDS)
|
||||
def milli = Duration(d, MILLISECONDS)
|
||||
|
||||
def seconds = Duration(d, SECONDS)
|
||||
def second = Duration(d, SECONDS)
|
||||
|
||||
def minutes = Duration(d, MINUTES)
|
||||
def minute = Duration(d, MINUTES)
|
||||
|
||||
def hours = Duration(d, HOURS)
|
||||
def hour = Duration(d, HOURS)
|
||||
|
||||
def days = Duration(d, DAYS)
|
||||
def day = Duration(d, DAYS)
|
||||
|
||||
def nanoseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, NANOSECONDS))
|
||||
def nanos[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, NANOSECONDS))
|
||||
def nanosecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, NANOSECONDS))
|
||||
def nano[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, NANOSECONDS))
|
||||
|
||||
def microseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MICROSECONDS))
|
||||
def micros[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MICROSECONDS))
|
||||
def microsecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MICROSECONDS))
|
||||
def micro[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MICROSECONDS))
|
||||
|
||||
def milliseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MILLISECONDS))
|
||||
def millis[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MILLISECONDS))
|
||||
def millisecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MILLISECONDS))
|
||||
def milli[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MILLISECONDS))
|
||||
|
||||
def seconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, SECONDS))
|
||||
def second[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, SECONDS))
|
||||
|
||||
def minutes[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MINUTES))
|
||||
def minute[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MINUTES))
|
||||
|
||||
def hours[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, HOURS))
|
||||
def hour[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, HOURS))
|
||||
|
||||
def days[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, DAYS))
|
||||
def day[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, DAYS))
|
||||
class DurationDouble(d: Double) extends DurationOps {
|
||||
override protected def from(timeUnit: TimeUnit): FiniteDuration = Duration(d, timeUnit)
|
||||
}
|
||||
|
||||
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
|
||||
|
|
@ -565,24 +450,27 @@ case class Timeout(duration: Duration) {
|
|||
def this(length: Long, unit: TimeUnit) = this(Duration(length, unit))
|
||||
}
|
||||
|
||||
/**
|
||||
* A Timeout is a wrapper on top of Duration to be more specific about what the duration means.
|
||||
*/
|
||||
object Timeout {
|
||||
|
||||
/**
|
||||
* A timeout with zero duration, will cause most requests to always timeout.
|
||||
*/
|
||||
val zero = new Timeout(Duration.Zero)
|
||||
val zero: Timeout = new Timeout(Duration.Zero)
|
||||
|
||||
/**
|
||||
* A Timeout with infinite duration. Will never timeout. Use extreme caution with this
|
||||
* as it may cause memory leaks, blocked threads, or may not even be supported by
|
||||
* the receiver, which would result in an exception.
|
||||
*/
|
||||
val never = new Timeout(Duration.Inf)
|
||||
val never: Timeout = new Timeout(Duration.Inf)
|
||||
|
||||
def apply(timeout: Long) = new Timeout(timeout)
|
||||
def apply(length: Long, unit: TimeUnit) = new Timeout(length, unit)
|
||||
def apply(timeout: Long): Timeout = new Timeout(timeout)
|
||||
def apply(length: Long, unit: TimeUnit): Timeout = new Timeout(length, unit)
|
||||
|
||||
implicit def durationToTimeout(duration: Duration) = new Timeout(duration)
|
||||
implicit def intToTimeout(timeout: Int) = new Timeout(timeout)
|
||||
implicit def longToTimeout(timeout: Long) = new Timeout(timeout)
|
||||
implicit def durationToTimeout(duration: Duration): Timeout = new Timeout(duration)
|
||||
implicit def intToTimeout(timeout: Int): Timeout = new Timeout(timeout)
|
||||
implicit def longToTimeout(timeout: Long): Timeout = new Timeout(timeout)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -45,18 +45,13 @@ object Helpers {
|
|||
else base64(next, sb)
|
||||
}
|
||||
|
||||
def ignore[E: Manifest](body: ⇒ Unit) {
|
||||
try {
|
||||
body
|
||||
} catch {
|
||||
case e if manifest[E].erasure.isAssignableFrom(e.getClass) ⇒ ()
|
||||
}
|
||||
}
|
||||
//FIXME docs
|
||||
def ignore[E: Manifest](body: ⇒ Unit): Unit =
|
||||
try body catch { case e if manifest[E].erasure.isAssignableFrom(e.getClass) ⇒ () }
|
||||
|
||||
def withPrintStackTraceOnError(body: ⇒ Unit) {
|
||||
try {
|
||||
body
|
||||
} catch {
|
||||
//FIXME docs
|
||||
def withPrintStackTraceOnError(body: ⇒ Unit): Unit = {
|
||||
try body catch {
|
||||
case e: Throwable ⇒
|
||||
val sw = new java.io.StringWriter()
|
||||
var root = e
|
||||
|
|
|
|||
|
|
@ -91,7 +91,7 @@ class Index[K, V](val mapSize: Int, val valueComparator: Comparator[V]) {
|
|||
/**
|
||||
* Applies the supplied function to all keys and their values
|
||||
*/
|
||||
def foreach(fun: (K, V) ⇒ Unit) {
|
||||
def foreach(fun: (K, V) ⇒ Unit): Unit = {
|
||||
import scala.collection.JavaConversions._
|
||||
container.entrySet foreach { e ⇒ e.getValue.foreach(fun(e.getKey, _)) }
|
||||
}
|
||||
|
|
@ -112,7 +112,7 @@ class Index[K, V](val mapSize: Int, val valueComparator: Comparator[V]) {
|
|||
/**
|
||||
* Returns the key set.
|
||||
*/
|
||||
def keys = scala.collection.JavaConversions.collectionAsScalaIterable(container.keySet)
|
||||
def keys: Iterable[K] = scala.collection.JavaConversions.collectionAsScalaIterable(container.keySet)
|
||||
|
||||
/**
|
||||
* Disassociates the value of type V from the key of type K
|
||||
|
|
|
|||
|
|
@ -7,17 +7,12 @@ package akka.util
|
|||
import java.util.concurrent.locks.{ ReentrantLock }
|
||||
import java.util.concurrent.atomic.{ AtomicBoolean }
|
||||
|
||||
final class ReentrantGuard {
|
||||
final val lock = new ReentrantLock
|
||||
final class ReentrantGuard extends ReentrantLock {
|
||||
|
||||
@inline
|
||||
final def withGuard[T](body: ⇒ T): T = {
|
||||
lock.lock
|
||||
try {
|
||||
body
|
||||
} finally {
|
||||
lock.unlock
|
||||
}
|
||||
lock()
|
||||
try body finally unlock()
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -29,9 +24,7 @@ class Switch(startAsOn: Boolean = false) {
|
|||
|
||||
protected def transcend(from: Boolean, action: ⇒ Unit): Boolean = synchronized {
|
||||
if (switch.compareAndSet(from, !from)) {
|
||||
try {
|
||||
action
|
||||
} catch {
|
||||
try action catch {
|
||||
case e ⇒
|
||||
switch.compareAndSet(!from, from) // revert status
|
||||
throw e
|
||||
|
|
@ -67,18 +60,12 @@ class Switch(startAsOn: Boolean = false) {
|
|||
/**
|
||||
* Executes the provided action and returns its value if the switch is IMMEDIATELY on (i.e. no lock involved)
|
||||
*/
|
||||
def ifOnYield[T](action: ⇒ T): Option[T] = {
|
||||
if (switch.get) Some(action)
|
||||
else None
|
||||
}
|
||||
def ifOnYield[T](action: ⇒ T): Option[T] = if (switch.get) Some(action) else None
|
||||
|
||||
/**
|
||||
* Executes the provided action and returns its value if the switch is IMMEDIATELY off (i.e. no lock involved)
|
||||
*/
|
||||
def ifOffYield[T](action: ⇒ T): Option[T] = {
|
||||
if (!switch.get) Some(action)
|
||||
else None
|
||||
}
|
||||
def ifOffYield[T](action: ⇒ T): Option[T] = if (!switch.get) Some(action) else None
|
||||
|
||||
/**
|
||||
* Executes the provided action and returns if the action was executed or not, if the switch is IMMEDIATELY on (i.e. no lock involved)
|
||||
|
|
@ -104,19 +91,13 @@ class Switch(startAsOn: Boolean = false) {
|
|||
* Executes the provided action and returns its value if the switch is on, waiting for any pending changes to happen before (locking)
|
||||
* Be careful of longrunning or blocking within the provided action as it can lead to deadlocks or bad performance
|
||||
*/
|
||||
def whileOnYield[T](action: ⇒ T): Option[T] = synchronized {
|
||||
if (switch.get) Some(action)
|
||||
else None
|
||||
}
|
||||
def whileOnYield[T](action: ⇒ T): Option[T] = synchronized { if (switch.get) Some(action) else None }
|
||||
|
||||
/**
|
||||
* Executes the provided action and returns its value if the switch is off, waiting for any pending changes to happen before (locking)
|
||||
* Be careful of longrunning or blocking within the provided action as it can lead to deadlocks or bad performance
|
||||
*/
|
||||
def whileOffYield[T](action: ⇒ T): Option[T] = synchronized {
|
||||
if (!switch.get) Some(action)
|
||||
else None
|
||||
}
|
||||
def whileOffYield[T](action: ⇒ T): Option[T] = synchronized { if (!switch.get) Some(action) else None }
|
||||
|
||||
/**
|
||||
* Executes the provided action and returns if the action was executed or not, if the switch is on, waiting for any pending changes to happen before (locking)
|
||||
|
|
@ -144,22 +125,20 @@ class Switch(startAsOn: Boolean = false) {
|
|||
* Executes the provided callbacks depending on if the switch is either on or off waiting for any pending changes to happen before (locking)
|
||||
* Be careful of longrunning or blocking within the provided action as it can lead to deadlocks or bad performance
|
||||
*/
|
||||
def fold[T](on: ⇒ T)(off: ⇒ T) = synchronized {
|
||||
if (switch.get) on else off
|
||||
}
|
||||
def fold[T](on: ⇒ T)(off: ⇒ T): T = synchronized { if (switch.get) on else off }
|
||||
|
||||
/**
|
||||
* Executes the given code while holding this switch’s lock, i.e. protected from concurrent modification of the switch status.
|
||||
*/
|
||||
def locked[T](code: ⇒ T) = synchronized { code }
|
||||
def locked[T](code: ⇒ T): T = synchronized { code }
|
||||
|
||||
/**
|
||||
* Returns whether the switch is IMMEDIATELY on (no locking)
|
||||
*/
|
||||
def isOn = switch.get
|
||||
def isOn: Boolean = switch.get
|
||||
|
||||
/**
|
||||
* Returns whether the switch is IMMEDDIATELY off (no locking)
|
||||
*/
|
||||
def isOff = !isOn
|
||||
def isOff: Boolean = !isOn
|
||||
}
|
||||
|
|
|
|||
|
|
@ -6,8 +6,10 @@ package akka.util
|
|||
/**
|
||||
* Collection of internal reflection utilities which may or may not be
|
||||
* available (most services specific to HotSpot, but fails gracefully).
|
||||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
object Reflect {
|
||||
private[akka] object Reflect {
|
||||
|
||||
/**
|
||||
* This optionally holds a function which looks N levels above itself
|
||||
|
|
|
|||
|
|
@ -7,6 +7,9 @@ package akka.util;
|
|||
|
||||
import java.lang.reflect.Field;
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
public final class Unsafe {
|
||||
public final static sun.misc.Unsafe instance;
|
||||
static {
|
||||
|
|
|
|||
|
|
@ -7,6 +7,7 @@ package akka.util
|
|||
import scala.util.continuations._
|
||||
import akka.dispatch.MessageDispatcher
|
||||
|
||||
//FIXME Needs docs
|
||||
package object cps {
|
||||
def matchC[A, B, C, D](in: A)(pf: PartialFunction[A, B @cpsParam[C, D]]): B @cpsParam[C, D] = pf(in)
|
||||
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@
|
|||
package akka.util
|
||||
|
||||
import java.util.concurrent.TimeUnit
|
||||
|
||||
//FIXME Needs docs
|
||||
package object duration {
|
||||
trait Classifier[C] {
|
||||
type R
|
||||
|
|
@ -15,38 +15,32 @@ package object duration {
|
|||
object span
|
||||
implicit object spanConvert extends Classifier[span.type] {
|
||||
type R = FiniteDuration
|
||||
def convert(d: FiniteDuration) = d
|
||||
def convert(d: FiniteDuration): FiniteDuration = d
|
||||
}
|
||||
|
||||
object fromNow
|
||||
implicit object fromNowConvert extends Classifier[fromNow.type] {
|
||||
type R = Deadline
|
||||
def convert(d: FiniteDuration) = Deadline.now + d
|
||||
def convert(d: FiniteDuration): Deadline = Deadline.now + d
|
||||
}
|
||||
|
||||
implicit def intToDurationInt(n: Int) = new DurationInt(n)
|
||||
implicit def longToDurationLong(n: Long) = new DurationLong(n)
|
||||
implicit def doubleToDurationDouble(d: Double) = new DurationDouble(d)
|
||||
implicit def intToDurationInt(n: Int): DurationInt = new DurationInt(n)
|
||||
implicit def longToDurationLong(n: Long): DurationLong = new DurationLong(n)
|
||||
implicit def doubleToDurationDouble(d: Double): DurationDouble = new DurationDouble(d)
|
||||
|
||||
implicit def pairIntToDuration(p: (Int, TimeUnit)) = Duration(p._1, p._2)
|
||||
implicit def pairLongToDuration(p: (Long, TimeUnit)) = Duration(p._1, p._2)
|
||||
implicit def durationToPair(d: Duration) = (d.length, d.unit)
|
||||
implicit def pairIntToDuration(p: (Int, TimeUnit)): FiniteDuration = Duration(p._1, p._2)
|
||||
implicit def pairLongToDuration(p: (Long, TimeUnit)): FiniteDuration = Duration(p._1, p._2)
|
||||
implicit def durationToPair(d: Duration): (Long, TimeUnit) = (d.length, d.unit)
|
||||
|
||||
/*
|
||||
* avoid reflection based invocation by using non-duck type
|
||||
*/
|
||||
class IntMult(i: Int) {
|
||||
def *(d: Duration) = d * i
|
||||
}
|
||||
implicit def intMult(i: Int) = new IntMult(i)
|
||||
class IntMult(i: Int) { def *(d: Duration): Duration = d * i }
|
||||
implicit def intMult(i: Int): IntMult = new IntMult(i)
|
||||
|
||||
class LongMult(l: Long) {
|
||||
def *(d: Duration) = d * l
|
||||
}
|
||||
implicit def longMult(l: Long) = new LongMult(l)
|
||||
class LongMult(l: Long) { def *(d: Duration): Duration = d * l }
|
||||
implicit def longMult(l: Long): LongMult = new LongMult(l)
|
||||
|
||||
class DoubleMult(f: Double) {
|
||||
def *(d: Duration) = d * f
|
||||
}
|
||||
implicit def doubleMult(f: Double) = new DoubleMult(f)
|
||||
class DoubleMult(f: Double) { def *(d: Duration): Duration = d * f }
|
||||
implicit def doubleMult(f: Double): DoubleMult = new DoubleMult(f)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -96,18 +96,18 @@ object Agent {
|
|||
* }}}
|
||||
*/
|
||||
class Agent[T](initialValue: T, system: ActorSystem) {
|
||||
private[akka] val ref = Ref(initialValue)
|
||||
private[akka] val updater = system.actorOf(Props(new AgentUpdater(this))).asInstanceOf[LocalActorRef] //TODO can we avoid this somehow?
|
||||
private val ref = Ref(initialValue)
|
||||
private val updater = system.actorOf(Props(new AgentUpdater(this, ref))).asInstanceOf[LocalActorRef] //TODO can we avoid this somehow?
|
||||
|
||||
/**
|
||||
* Read the internal state of the agent.
|
||||
*/
|
||||
def get() = ref.single.get
|
||||
def get(): T = ref.single.get
|
||||
|
||||
/**
|
||||
* Read the internal state of the agent.
|
||||
*/
|
||||
def apply() = get
|
||||
def apply(): T = get
|
||||
|
||||
/**
|
||||
* Dispatch a function to update the internal state.
|
||||
|
|
@ -154,7 +154,7 @@ class Agent[T](initialValue: T, system: ActorSystem) {
|
|||
def sendOff(f: T ⇒ T): Unit = {
|
||||
send((value: T) ⇒ {
|
||||
suspend()
|
||||
val threadBased = system.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher("akka.agent.send-off-dispatcher"))
|
||||
val threadBased = system.actorOf(Props(new ThreadBasedAgentUpdater(this, ref)).withDispatcher("akka.agent.send-off-dispatcher"))
|
||||
threadBased ! Update(f)
|
||||
value
|
||||
})
|
||||
|
|
@ -171,7 +171,7 @@ class Agent[T](initialValue: T, system: ActorSystem) {
|
|||
val result = Promise[T]()(system.dispatcher)
|
||||
send((value: T) ⇒ {
|
||||
suspend()
|
||||
val threadBased = system.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher("akka.agent.alter-off-dispatcher"))
|
||||
val threadBased = system.actorOf(Props(new ThreadBasedAgentUpdater(this, ref)).withDispatcher("akka.agent.alter-off-dispatcher"))
|
||||
result completeWith ask(threadBased, Alter(f))(timeout).asInstanceOf[Future[T]]
|
||||
value
|
||||
})
|
||||
|
|
@ -209,18 +209,18 @@ class Agent[T](initialValue: T, system: ActorSystem) {
|
|||
/**
|
||||
* Suspends processing of `send` actions for the agent.
|
||||
*/
|
||||
def suspend() = updater.suspend()
|
||||
def suspend(): Unit = updater.suspend()
|
||||
|
||||
/**
|
||||
* Resumes processing of `send` actions for the agent.
|
||||
*/
|
||||
def resume() = updater.resume()
|
||||
def resume(): Unit = updater.resume()
|
||||
|
||||
/**
|
||||
* Closes the agents and makes it eligible for garbage collection.
|
||||
* A closed agent cannot accept any `send` actions.
|
||||
*/
|
||||
def close() = updater.stop()
|
||||
def close(): Unit = updater.stop()
|
||||
|
||||
// ---------------------------------------------
|
||||
// Support for Java API Functions and Procedures
|
||||
|
|
@ -281,8 +281,10 @@ class Agent[T](initialValue: T, system: ActorSystem) {
|
|||
|
||||
/**
|
||||
* Agent updater actor. Used internally for `send` actions.
|
||||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
class AgentUpdater[T](agent: Agent[T]) extends Actor {
|
||||
private[akka] class AgentUpdater[T](agent: Agent[T], ref: Ref[T]) extends Actor {
|
||||
def receive = {
|
||||
case u: Update[_] ⇒ update(u.function.asInstanceOf[T ⇒ T])
|
||||
case a: Alter[_] ⇒ sender ! update(a.function.asInstanceOf[T ⇒ T])
|
||||
|
|
@ -290,13 +292,15 @@ class AgentUpdater[T](agent: Agent[T]) extends Actor {
|
|||
case _ ⇒
|
||||
}
|
||||
|
||||
def update(function: T ⇒ T): T = agent.ref.single.transformAndGet(function)
|
||||
def update(function: T ⇒ T): T = ref.single.transformAndGet(function)
|
||||
}
|
||||
|
||||
/**
|
||||
* Thread-based agent updater actor. Used internally for `sendOff` actions.
|
||||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
class ThreadBasedAgentUpdater[T](agent: Agent[T]) extends Actor {
|
||||
private[akka] class ThreadBasedAgentUpdater[T](agent: Agent[T], ref: Ref[T]) extends Actor {
|
||||
def receive = {
|
||||
case u: Update[_] ⇒ try {
|
||||
update(u.function.asInstanceOf[T ⇒ T])
|
||||
|
|
@ -313,5 +317,5 @@ class ThreadBasedAgentUpdater[T](agent: Agent[T]) extends Actor {
|
|||
case _ ⇒ context.stop(self)
|
||||
}
|
||||
|
||||
def update(function: T ⇒ T): T = agent.ref.single.transformAndGet(function)
|
||||
def update(function: T ⇒ T): T = ref.single.transformAndGet(function)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -18,9 +18,9 @@ import akka.pattern._
|
|||
trait Activation {
|
||||
import akka.dispatch.Await
|
||||
|
||||
def system: ActorSystem
|
||||
def system: ActorSystem //FIXME Why is this here, what's it needed for and who should use it?
|
||||
|
||||
private val activationTracker = system.actorOf(Props[ActivationTracker], "camelActivationTracker")
|
||||
private val activationTracker = system.actorOf(Props[ActivationTracker], "camelActivationTracker") //FIXME Why is this also top level?
|
||||
|
||||
/**
|
||||
* Awaits for endpoint to be activated. It blocks until the endpoint is registered in camel context or timeout expires.
|
||||
|
|
@ -29,13 +29,10 @@ trait Activation {
|
|||
* @throws akka.camel.ActivationTimeoutException if endpoint is not activated within timeout.
|
||||
* @return the activated ActorRef
|
||||
*/
|
||||
def awaitActivation(endpoint: ActorRef, timeout: Duration): ActorRef = {
|
||||
try {
|
||||
Await.result(activationFutureFor(endpoint, timeout), timeout)
|
||||
} catch {
|
||||
def awaitActivation(endpoint: ActorRef, timeout: Duration): ActorRef =
|
||||
try Await.result(activationFutureFor(endpoint, timeout), timeout) catch {
|
||||
case e: TimeoutException ⇒ throw new ActivationTimeoutException(endpoint, timeout)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Awaits for endpoint to be de-activated. It is blocking until endpoint is unregistered in camel context or timeout expires.
|
||||
|
|
@ -43,37 +40,32 @@ trait Activation {
|
|||
* @param timeout the timeout for the wait
|
||||
* @throws akka.camel.DeActivationTimeoutException if endpoint is not de-activated within timeout.
|
||||
*/
|
||||
def awaitDeactivation(endpoint: ActorRef, timeout: Duration) {
|
||||
try {
|
||||
Await.result(deactivationFutureFor(endpoint, timeout), timeout)
|
||||
} catch {
|
||||
def awaitDeactivation(endpoint: ActorRef, timeout: Duration): Unit =
|
||||
try Await.result(deactivationFutureFor(endpoint, timeout), timeout) catch {
|
||||
case e: TimeoutException ⇒ throw new DeActivationTimeoutException(endpoint, timeout)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Similar to `awaitActivation` but returns a future instead.
|
||||
* @param endpoint the endpoint to be activated
|
||||
* @param timeout the timeout for the Future
|
||||
*/
|
||||
def activationFutureFor(endpoint: ActorRef, timeout: Duration): Future[ActorRef] = {
|
||||
def activationFutureFor(endpoint: ActorRef, timeout: Duration): Future[ActorRef] =
|
||||
(activationTracker.ask(AwaitActivation(endpoint))(Timeout(timeout))).map[ActorRef] {
|
||||
case EndpointActivated(_) ⇒ endpoint
|
||||
case EndpointFailedToActivate(_, cause) ⇒ throw cause
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Similar to awaitDeactivation but returns a future instead.
|
||||
* @param endpoint the endpoint to be deactivated
|
||||
* @param timeout the timeout of the Future
|
||||
*/
|
||||
def deactivationFutureFor(endpoint: ActorRef, timeout: Duration): Future[Unit] = {
|
||||
def deactivationFutureFor(endpoint: ActorRef, timeout: Duration): Future[Unit] =
|
||||
(activationTracker.ask(AwaitDeActivation(endpoint))(Timeout(timeout))).map[Unit] {
|
||||
case EndpointDeActivated(_) ⇒ ()
|
||||
case EndpointFailedToDeActivate(_, cause) ⇒ throw cause
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -82,7 +74,7 @@ trait Activation {
|
|||
* @param timeout the timeout
|
||||
*/
|
||||
class DeActivationTimeoutException(endpoint: ActorRef, timeout: Duration) extends TimeoutException {
|
||||
override def getMessage = "Timed out after %s, while waiting for de-activation of %s" format (timeout, endpoint.path)
|
||||
override def getMessage: String = "Timed out after %s, while waiting for de-activation of %s" format (timeout, endpoint.path)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -91,5 +83,5 @@ class DeActivationTimeoutException(endpoint: ActorRef, timeout: Duration) extend
|
|||
* @param timeout the timeout
|
||||
*/
|
||||
class ActivationTimeoutException(endpoint: ActorRef, timeout: Duration) extends TimeoutException {
|
||||
override def getMessage = "Timed out after %s, while waiting for activation of %s" format (timeout, endpoint.path)
|
||||
override def getMessage: String = "Timed out after %s, while waiting for activation of %s" format (timeout, endpoint.path)
|
||||
}
|
||||
|
|
@ -6,5 +6,5 @@ package akka.camel
|
|||
* @author Martin Krasser
|
||||
*/
|
||||
class ActorNotRegisteredException(uri: String) extends RuntimeException {
|
||||
override def getMessage = "Actor [%s] doesn't exist" format uri
|
||||
override def getMessage: String = "Actor [%s] doesn't exist" format uri
|
||||
}
|
||||
|
|
|
|||
|
|
@ -29,7 +29,8 @@ class ActorRouteDefinition(definition: ProcessorDefinition[_]) {
|
|||
* @param actorRef the consumer with a default configuration.
|
||||
* @return the path to the actor, as a camel uri String
|
||||
*/
|
||||
def to(actorRef: ActorRef) = definition.to(ActorEndpointPath(actorRef).toCamelPath())
|
||||
def to(actorRef: ActorRef) = //FIXME What is the return type of this?
|
||||
definition.to(ActorEndpointPath(actorRef).toCamelPath())
|
||||
|
||||
/**
|
||||
* Sends the message to an ActorRef endpoint
|
||||
|
|
@ -37,6 +38,7 @@ class ActorRouteDefinition(definition: ProcessorDefinition[_]) {
|
|||
* @param consumerConfig the configuration for the consumer
|
||||
* @return the path to the actor, as a camel uri String
|
||||
*/
|
||||
def to(actorRef: ActorRef, consumerConfig: ConsumerConfig) = definition.to(ActorEndpointPath(actorRef).toCamelPath(consumerConfig))
|
||||
def to(actorRef: ActorRef, consumerConfig: ConsumerConfig) = //FIXME What is the return type of this?
|
||||
definition.to(ActorEndpointPath(actorRef).toCamelPath(consumerConfig))
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -50,13 +50,13 @@ object CamelExtension extends ExtensionId[Camel] with ExtensionIdProvider {
|
|||
/**
|
||||
* Creates a new instance of Camel and makes sure it gets stopped when the actor system is shutdown.
|
||||
*/
|
||||
def createExtension(system: ExtendedActorSystem) = {
|
||||
override def createExtension(system: ExtendedActorSystem): Camel = {
|
||||
val camel = new DefaultCamel(system).start
|
||||
system.registerOnTermination(camel.shutdown())
|
||||
camel
|
||||
}
|
||||
|
||||
def lookup(): ExtensionId[Camel] = CamelExtension
|
||||
override def lookup(): ExtensionId[Camel] = CamelExtension
|
||||
|
||||
override def get(system: ActorSystem): Camel = super.get(system)
|
||||
}
|
||||
|
|
@ -21,12 +21,12 @@ case class CamelMessage(body: Any, headers: Map[String, Any]) {
|
|||
|
||||
def this(body: Any, headers: JMap[String, Any]) = this(body, headers.toMap) //for Java
|
||||
|
||||
override def toString = "CamelMessage(%s, %s)" format (body, headers)
|
||||
override def toString: String = "CamelMessage(%s, %s)" format (body, headers)
|
||||
|
||||
/**
|
||||
* Returns those headers from this message whose name is contained in <code>names</code>.
|
||||
*/
|
||||
def headers(names: Set[String]): Map[String, Any] = headers.filterKeys(names contains _)
|
||||
def headers(names: Set[String]): Map[String, Any] = headers filterKeys names
|
||||
|
||||
/**
|
||||
* Returns those headers from this message whose name is contained in <code>names</code>.
|
||||
|
|
@ -75,7 +75,7 @@ case class CamelMessage(body: Any, headers: Map[String, Any]) {
|
|||
/**
|
||||
* Creates a CamelMessage with a given <code>body</code>.
|
||||
*/
|
||||
def withBody(body: Any) = CamelMessage(body, this.headers)
|
||||
def withBody(body: Any): CamelMessage = CamelMessage(body, this.headers)
|
||||
|
||||
/**
|
||||
* Creates a new CamelMessage with given <code>headers</code>.
|
||||
|
|
@ -119,9 +119,9 @@ case class CamelMessage(body: Any, headers: Map[String, Any]) {
|
|||
* Creates a new CamelMessage where the header with given <code>headerName</code> is removed from
|
||||
* the existing headers.
|
||||
*/
|
||||
def withoutHeader(headerName: String) = copy(this.body, this.headers - headerName)
|
||||
def withoutHeader(headerName: String): CamelMessage = copy(this.body, this.headers - headerName)
|
||||
|
||||
def copyContentTo(to: JCamelMessage) = {
|
||||
def copyContentTo(to: JCamelMessage): Unit = {
|
||||
to.setBody(this.body)
|
||||
for ((name, value) ← this.headers) to.getHeaders.put(name, value.asInstanceOf[AnyRef])
|
||||
}
|
||||
|
|
@ -145,8 +145,7 @@ case class CamelMessage(body: Any, headers: Map[String, Any]) {
|
|||
* Java API
|
||||
*
|
||||
*/
|
||||
def getBodyAs[T](clazz: Class[T], camelContext: CamelContext): T =
|
||||
camelContext.getTypeConverter.mandatoryConvertTo[T](clazz, body)
|
||||
def getBodyAs[T](clazz: Class[T], camelContext: CamelContext): T = camelContext.getTypeConverter.mandatoryConvertTo[T](clazz, body)
|
||||
|
||||
/**
|
||||
* Creates a CamelMessage with current <code>body</code> converted to type <code>T</code>.
|
||||
|
|
@ -184,7 +183,7 @@ case class CamelMessage(body: Any, headers: Map[String, Any]) {
|
|||
* <p>
|
||||
* Java API
|
||||
*/
|
||||
def getHeaderAs[T](name: String, clazz: Class[T], camelContext: CamelContext) = headerAs[T](name)(Manifest.classType(clazz), camelContext).get
|
||||
def getHeaderAs[T](name: String, clazz: Class[T], camelContext: CamelContext): T = headerAs[T](name)(Manifest.classType(clazz), camelContext).get
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -201,7 +200,7 @@ object CamelMessage {
|
|||
* so that it can be correlated with an asynchronous response. Messages send to Consumer
|
||||
* actors have this header already set.
|
||||
*/
|
||||
val MessageExchangeId = "MessageExchangeId".intern
|
||||
val MessageExchangeId = "MessageExchangeId".intern //Deliberately without type ascription to make it a constant
|
||||
|
||||
/**
|
||||
* Creates a canonical form of the given message <code>msg</code>. If <code>msg</code> of type
|
||||
|
|
@ -244,5 +243,7 @@ case object Ack {
|
|||
* message or Exchange.getOut message, depending on the exchange pattern.
|
||||
*
|
||||
*/
|
||||
class AkkaCamelException private[akka] (cause: Throwable, val headers: Map[String, Any] = Map.empty)
|
||||
extends AkkaException(cause.getMessage, cause)
|
||||
class AkkaCamelException private[akka] (cause: Throwable, val headers: Map[String, Any])
|
||||
extends AkkaException(cause.getMessage, cause) {
|
||||
def this(cause: Throwable) = this(cause, Map.empty)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -31,7 +31,7 @@ trait ConsumerConfig {
|
|||
/**
|
||||
* How long the actor should wait for activation before it fails.
|
||||
*/
|
||||
def activationTimeout: Duration = 10 seconds
|
||||
def activationTimeout: Duration = 10 seconds // FIXME Should be configured in reference.conf
|
||||
|
||||
/**
|
||||
* When endpoint is out-capable (can produce responses) replyTimeout is the maximum time
|
||||
|
|
@ -39,14 +39,14 @@ trait ConsumerConfig {
|
|||
* This setting is used for out-capable, in-only, manually acknowledged communication.
|
||||
* When the blocking is set to Blocking replyTimeout is ignored.
|
||||
*/
|
||||
def replyTimeout: Duration = 1 minute
|
||||
def replyTimeout: Duration = 1 minute // FIXME Should be configured in reference.conf
|
||||
|
||||
/**
|
||||
* Determines whether one-way communications between an endpoint and this consumer actor
|
||||
* should be auto-acknowledged or application-acknowledged.
|
||||
* This flag has only effect when exchange is in-only.
|
||||
*/
|
||||
def autoack: Boolean = true
|
||||
def autoack: Boolean = true // FIXME Should be configured in reference.conf
|
||||
|
||||
/**
|
||||
* The route definition handler for creating a custom route to this consumer instance.
|
||||
|
|
|
|||
|
|
@ -6,8 +6,9 @@ package akka.camel
|
|||
|
||||
import akka.actor.Actor
|
||||
import internal.CamelExchangeAdapter
|
||||
import org.apache.camel.{ Exchange, ExchangePattern, AsyncCallback }
|
||||
import akka.actor.Status.Failure
|
||||
import org.apache.camel.{ Endpoint, Exchange, ExchangePattern, AsyncCallback }
|
||||
import org.apache.camel.processor.SendProcessor
|
||||
|
||||
/**
|
||||
* Support trait for producing messages to Camel endpoints.
|
||||
|
|
@ -15,19 +16,19 @@ import akka.actor.Status.Failure
|
|||
* @author Martin Krasser
|
||||
*/
|
||||
trait ProducerSupport { this: Actor ⇒
|
||||
protected[this] implicit def camel = CamelExtension(context.system)
|
||||
protected[this] implicit def camel = CamelExtension(context.system) // FIXME This is duplicated from Consumer, create a common base-trait?
|
||||
|
||||
/**
|
||||
* camelContext implicit is useful when using advanced methods of CamelMessage.
|
||||
*/
|
||||
protected[this] implicit def camelContext = camel.context
|
||||
protected[this] implicit def camelContext = camel.context // FIXME This is duplicated from Consumer, create a common base-trait?
|
||||
|
||||
protected[this] lazy val (endpoint, processor) = camel.registerProducer(self, endpointUri)
|
||||
protected[this] lazy val (endpoint: Endpoint, processor: SendProcessor) = camel.registerProducer(self, endpointUri)
|
||||
|
||||
/**
|
||||
* CamelMessage headers to copy by default from request message to response-message.
|
||||
*/
|
||||
private val headersToCopyDefault = Set(CamelMessage.MessageExchangeId)
|
||||
private val headersToCopyDefault: Set[String] = Set(CamelMessage.MessageExchangeId)
|
||||
|
||||
/**
|
||||
* If set to false (default), this producer expects a response message from the Camel endpoint.
|
||||
|
|
@ -64,20 +65,21 @@ trait ProducerSupport { this: Actor ⇒
|
|||
* @param pattern exchange pattern
|
||||
*/
|
||||
protected def produce(msg: Any, pattern: ExchangePattern): Unit = {
|
||||
implicit def toExchangeAdapter(exchange: Exchange): CamelExchangeAdapter = new CamelExchangeAdapter(exchange)
|
||||
// Need copies of sender reference here since the callback could be done
|
||||
// later by another thread.
|
||||
val producer = self
|
||||
val originalSender = sender
|
||||
|
||||
val cmsg = CamelMessage.canonicalize(msg)
|
||||
val exchange = endpoint.createExchange(pattern)
|
||||
exchange.setRequest(cmsg)
|
||||
processor.process(exchange, new AsyncCallback {
|
||||
val producer = self
|
||||
// Need copies of sender reference here since the callback could be done
|
||||
// later by another thread.
|
||||
val originalSender = sender
|
||||
val xchg = new CamelExchangeAdapter(endpoint.createExchange(pattern))
|
||||
|
||||
xchg.setRequest(cmsg)
|
||||
|
||||
processor.process(xchg.exchange, new AsyncCallback {
|
||||
// Ignoring doneSync, sending back async uniformly.
|
||||
def done(doneSync: Boolean): Unit = producer.tell(
|
||||
if (exchange.isFailed) exchange.toFailureResult(cmsg.headers(headersToCopy))
|
||||
else MessageResult(exchange.toResponseMessage(cmsg.headers(headersToCopy))), originalSender)
|
||||
if (xchg.exchange.isFailed) xchg.toFailureResult(cmsg.headers(headersToCopy))
|
||||
else MessageResult(xchg.toResponseMessage(cmsg.headers(headersToCopy))), originalSender)
|
||||
})
|
||||
}
|
||||
|
||||
|
|
@ -94,9 +96,7 @@ trait ProducerSupport { this: Actor ⇒
|
|||
val e = new AkkaCamelException(res.cause, res.headers)
|
||||
routeResponse(Failure(e))
|
||||
throw e
|
||||
case msg ⇒
|
||||
val exchangePattern = if (oneway) ExchangePattern.InOnly else ExchangePattern.InOut
|
||||
produce(transformOutgoingMessage(msg), exchangePattern)
|
||||
case msg ⇒ produce(transformOutgoingMessage(msg), if (oneway) ExchangePattern.InOnly else ExchangePattern.InOut)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -134,7 +134,7 @@ trait Producer extends ProducerSupport { this: Actor ⇒
|
|||
* Default implementation of Actor.receive. Any messages received by this actors
|
||||
* will be produced to the endpoint specified by <code>endpointUri</code>.
|
||||
*/
|
||||
protected def receive = produce
|
||||
def receive: Actor.Receive = produce
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -153,6 +153,6 @@ private case class FailureResult(cause: Throwable, headers: Map[String, Any] = M
|
|||
* @author Martin Krasser
|
||||
*/
|
||||
trait Oneway extends Producer { this: Actor ⇒
|
||||
override def oneway = true
|
||||
override def oneway: Boolean = true
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -20,7 +20,7 @@ private[camel] abstract class ActivationMessage(val actor: ActorRef)
|
|||
*
|
||||
*/
|
||||
private[camel] object ActivationMessage {
|
||||
def unapply(msg: ActivationMessage): Option[ActorRef] = Some(msg.actor)
|
||||
def unapply(msg: ActivationMessage): Option[ActorRef] = Option(msg.actor)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -96,17 +96,15 @@ private[akka] final class ActivationTracker extends Actor with ActorLogging {
|
|||
/**
|
||||
* Subscribes self to messages of type <code>ActivationMessage</code>
|
||||
*/
|
||||
override def preStart() {
|
||||
context.system.eventStream.subscribe(self, classOf[ActivationMessage])
|
||||
}
|
||||
override def preStart(): Unit = context.system.eventStream.subscribe(self, classOf[ActivationMessage])
|
||||
|
||||
override def receive = {
|
||||
case msg @ ActivationMessage(ref) ⇒
|
||||
val state = activations.getOrElseUpdate(ref, new ActivationStateMachine)
|
||||
(state.receive orElse logStateWarning(ref))(msg)
|
||||
(activations.getOrElseUpdate(ref, new ActivationStateMachine).receive orElse logStateWarning(ref))(msg)
|
||||
}
|
||||
|
||||
private[this] def logStateWarning(actorRef: ActorRef): Receive = { case msg ⇒ log.warning("Message [{}] not expected in current state of actor [{}]", msg, actorRef) }
|
||||
private[this] def logStateWarning(actorRef: ActorRef): Receive =
|
||||
{ case msg ⇒ log.warning("Message [{}] not expected in current state of actor [{}]", msg, actorRef) }
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -16,34 +16,34 @@ import akka.camel.{ FailureResult, AkkaCamelException, CamelMessage }
|
|||
*
|
||||
* @author Martin Krasser
|
||||
*/
|
||||
private[camel] class CamelExchangeAdapter(exchange: Exchange) {
|
||||
private[camel] class CamelExchangeAdapter(val exchange: Exchange) {
|
||||
/**
|
||||
* Returns the exchange id
|
||||
*/
|
||||
def getExchangeId = exchange.getExchangeId
|
||||
def getExchangeId: String = exchange.getExchangeId
|
||||
|
||||
/**
|
||||
* Returns if the exchange is out capable.
|
||||
*/
|
||||
def isOutCapable = exchange.getPattern.isOutCapable
|
||||
def isOutCapable: Boolean = exchange.getPattern.isOutCapable
|
||||
|
||||
/**
|
||||
* Sets Exchange.getIn from the given CamelMessage object.
|
||||
*/
|
||||
def setRequest(msg: CamelMessage) { msg.copyContentTo(request) }
|
||||
def setRequest(msg: CamelMessage): Unit = msg.copyContentTo(request)
|
||||
|
||||
/**
|
||||
* Depending on the exchange pattern, sets Exchange.getIn or Exchange.getOut from the given
|
||||
* CamelMessage object. If the exchange is out-capable then the Exchange.getOut is set, otherwise
|
||||
* Exchange.getIn.
|
||||
*/
|
||||
def setResponse(msg: CamelMessage) { msg.copyContentTo(response) }
|
||||
def setResponse(msg: CamelMessage): Unit = msg.copyContentTo(response)
|
||||
|
||||
/**
|
||||
* Sets Exchange.getException from the given FailureResult message. Headers of the FailureResult message
|
||||
* are ignored.
|
||||
*/
|
||||
def setFailure(msg: FailureResult) { exchange.setException(msg.cause) }
|
||||
def setFailure(msg: FailureResult): Unit = exchange.setException(msg.cause)
|
||||
|
||||
/**
|
||||
* Creates an immutable CamelMessage object from Exchange.getIn so it can be used with Actors.
|
||||
|
|
@ -120,7 +120,7 @@ private[camel] class CamelExchangeAdapter(exchange: Exchange) {
|
|||
*/
|
||||
def toResponseMessage(headers: Map[String, Any]): CamelMessage = CamelMessage.from(response, headers)
|
||||
|
||||
private def request = exchange.getIn
|
||||
private def request: JCamelMessage = exchange.getIn
|
||||
|
||||
private def response: JCamelMessage = ExchangeHelper.getResultMessage(exchange)
|
||||
|
||||
|
|
|
|||
|
|
@ -2,12 +2,12 @@ package akka.camel.internal
|
|||
|
||||
import akka.actor.ActorSystem
|
||||
import component.{ DurationTypeConverter, ActorComponent }
|
||||
import org.apache.camel.CamelContext
|
||||
import org.apache.camel.impl.DefaultCamelContext
|
||||
import scala.Predef._
|
||||
import akka.event.Logging
|
||||
import akka.camel.Camel
|
||||
import akka.util.{ NonFatal, Duration }
|
||||
import org.apache.camel.{ ProducerTemplate, CamelContext }
|
||||
|
||||
/**
|
||||
* For internal use only.
|
||||
|
|
@ -33,14 +33,14 @@ private[camel] class DefaultCamel(val system: ActorSystem) extends Camel {
|
|||
ctx
|
||||
}
|
||||
|
||||
lazy val template = context.createProducerTemplate()
|
||||
lazy val template: ProducerTemplate = context.createProducerTemplate()
|
||||
|
||||
/**
|
||||
* Starts camel and underlying camel context and template.
|
||||
* Only the creator of Camel should start and stop it.
|
||||
* @see akka.camel.DefaultCamel#stop()
|
||||
*/
|
||||
def start = {
|
||||
def start(): this.type = {
|
||||
context.start()
|
||||
try template.start() catch { case NonFatal(e) ⇒ context.stop(); throw e }
|
||||
log.debug("Started CamelContext[{}] for ActorSystem[{}]", context.getName, system.name)
|
||||
|
|
@ -54,9 +54,9 @@ private[camel] class DefaultCamel(val system: ActorSystem) extends Camel {
|
|||
*
|
||||
* @see akka.camel.DefaultCamel#start()
|
||||
*/
|
||||
def shutdown() {
|
||||
def shutdown(): Unit = {
|
||||
try context.stop() finally {
|
||||
try { template.stop() } catch { case NonFatal(e) ⇒ log.debug("Swallowing non-fatal exception [{}] on stopping Camel producer template", e) }
|
||||
try template.stop() catch { case NonFatal(e) ⇒ log.debug("Swallowing non-fatal exception [{}] on stopping Camel producer template", e) }
|
||||
}
|
||||
log.debug("Stopped CamelContext[{}] for ActorSystem[{}]", context.getName, system.name)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -11,6 +11,8 @@ import akka.util.NonFatal
|
|||
* Watches the end of life of <code>Producer</code>s.
|
||||
* Removes a <code>Producer</code> from the <code>ProducerRegistry</code> when it is <code>Terminated</code>,
|
||||
* which in turn stops the <code>SendProcessor</code>.
|
||||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
private class ProducerWatcher(registry: ProducerRegistry) extends Actor {
|
||||
override def receive = {
|
||||
|
|
@ -19,6 +21,9 @@ private class ProducerWatcher(registry: ProducerRegistry) extends Actor {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private case class RegisterProducer(actorRef: ActorRef)
|
||||
|
||||
/**
|
||||
|
|
@ -27,14 +32,11 @@ private case class RegisterProducer(actorRef: ActorRef)
|
|||
* Every <code>Producer</code> needs an <code>Endpoint</code> and a <code>SendProcessor</code>
|
||||
* to produce messages over an <code>Exchange</code>.
|
||||
*/
|
||||
private[camel] trait ProducerRegistry {
|
||||
this: Camel ⇒
|
||||
private[camel] trait ProducerRegistry { this: Camel ⇒
|
||||
private val camelObjects = new ConcurrentHashMap[ActorRef, (Endpoint, SendProcessor)]()
|
||||
private val watcher = system.actorOf(Props(new ProducerWatcher(this)))
|
||||
private val watcher = system.actorOf(Props(new ProducerWatcher(this))) //FIXME should this really be top level?
|
||||
|
||||
private def registerWatch(actorRef: ActorRef) {
|
||||
watcher ! RegisterProducer(actorRef)
|
||||
}
|
||||
private def registerWatch(actorRef: ActorRef): Unit = watcher ! RegisterProducer(actorRef)
|
||||
|
||||
/**
|
||||
* For internal use only.
|
||||
|
|
@ -77,7 +79,7 @@ private[camel] trait ProducerRegistry {
|
|||
case NonFatal(e) ⇒ {
|
||||
system.eventStream.publish(EndpointFailedToActivate(actorRef, e))
|
||||
// can't return null to the producer actor, so blow up actor in initialization.
|
||||
throw e
|
||||
throw e //FIXME I'm not a huge fan of log-rethrow, either log or rethrow
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -35,10 +35,8 @@ private[camel] class ActorComponent(camel: Camel) extends DefaultComponent {
|
|||
/**
|
||||
* @see org.apache.camel.Component
|
||||
*/
|
||||
def createEndpoint(uri: String, remaining: String, parameters: JMap[String, Object]): ActorEndpoint = {
|
||||
val path = ActorEndpointPath.fromCamelPath(remaining)
|
||||
new ActorEndpoint(uri, this, path, camel)
|
||||
}
|
||||
def createEndpoint(uri: String, remaining: String, parameters: JMap[String, Object]): ActorEndpoint =
|
||||
new ActorEndpoint(uri, this, ActorEndpointPath.fromCamelPath(remaining), camel)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -92,7 +90,7 @@ private[camel] class ActorEndpoint(uri: String,
|
|||
private[camel] trait ActorEndpointConfig {
|
||||
def path: ActorEndpointPath
|
||||
|
||||
@BeanProperty var replyTimeout: Duration = 1 minute
|
||||
@BeanProperty var replyTimeout: Duration = 1 minute // FIXME default should be in config, not code
|
||||
|
||||
/**
|
||||
* Whether to auto-acknowledge one-way message exchanges with (untyped) actors. This is
|
||||
|
|
@ -117,7 +115,7 @@ private[camel] class ActorProducer(val endpoint: ActorEndpoint, camel: Camel) ex
|
|||
* Calls the asynchronous version of the method and waits for the result (blocking).
|
||||
* @param exchange the exchange to process
|
||||
*/
|
||||
def process(exchange: Exchange) { processExchangeAdapter(new CamelExchangeAdapter(exchange)) }
|
||||
def process(exchange: Exchange): Unit = processExchangeAdapter(new CamelExchangeAdapter(exchange))
|
||||
|
||||
/**
|
||||
* Processes the message exchange. the caller supports having the exchange asynchronously processed.
|
||||
|
|
@ -129,13 +127,15 @@ private[camel] class ActorProducer(val endpoint: ActorEndpoint, camel: Camel) ex
|
|||
* The callback should therefore be careful of starting recursive loop.
|
||||
* @return (doneSync) true to continue execute synchronously, false to continue being executed asynchronously
|
||||
*/
|
||||
def process(exchange: Exchange, callback: AsyncCallback): Boolean = { processExchangeAdapter(new CamelExchangeAdapter(exchange), callback) }
|
||||
def process(exchange: Exchange, callback: AsyncCallback): Boolean = processExchangeAdapter(new CamelExchangeAdapter(exchange), callback)
|
||||
|
||||
/**
|
||||
* For internal use only. Processes the [[akka.camel.internal.CamelExchangeAdapter]]
|
||||
* @param exchange the [[akka.camel.internal.CamelExchangeAdapter]]
|
||||
*
|
||||
* WARNING UNBOUNDED BLOCKING AWAITS
|
||||
*/
|
||||
private[camel] def processExchangeAdapter(exchange: CamelExchangeAdapter) {
|
||||
private[camel] def processExchangeAdapter(exchange: CamelExchangeAdapter): Unit = {
|
||||
val isDone = new CountDownLatch(1)
|
||||
processExchangeAdapter(exchange, new AsyncCallback { def done(doneSync: Boolean) { isDone.countDown() } })
|
||||
isDone.await() // this should never wait forever as the process(exchange, callback) method guarantees that.
|
||||
|
|
@ -151,10 +151,10 @@ private[camel] class ActorProducer(val endpoint: ActorEndpoint, camel: Camel) ex
|
|||
private[camel] def processExchangeAdapter(exchange: CamelExchangeAdapter, callback: AsyncCallback): Boolean = {
|
||||
|
||||
// these notify methods are just a syntax sugar
|
||||
def notifyDoneSynchronously[A](a: A = null) = callback.done(true)
|
||||
def notifyDoneAsynchronously[A](a: A = null) = callback.done(false)
|
||||
def notifyDoneSynchronously[A](a: A = null): Unit = callback.done(true)
|
||||
def notifyDoneAsynchronously[A](a: A = null): Unit = callback.done(false)
|
||||
|
||||
def message = messageFor(exchange)
|
||||
def message: CamelMessage = messageFor(exchange)
|
||||
|
||||
if (exchange.isOutCapable) { //InOut
|
||||
sendAsync(message, onComplete = forwardResponseTo(exchange) andThen notifyDoneAsynchronously)
|
||||
|
|
@ -186,42 +186,41 @@ private[camel] class ActorProducer(val endpoint: ActorEndpoint, camel: Camel) ex
|
|||
|
||||
private def sendAsync(message: CamelMessage, onComplete: PartialFunction[Either[Throwable, Any], Unit]): Boolean = {
|
||||
try {
|
||||
val actor = actorFor(endpoint.path)
|
||||
val future = actor.ask(message)(new Timeout(endpoint.replyTimeout))
|
||||
future.onComplete(onComplete)
|
||||
actorFor(endpoint.path).ask(message)(Timeout(endpoint.replyTimeout)).onComplete(onComplete)
|
||||
} catch {
|
||||
case NonFatal(e) ⇒ onComplete(Left(e))
|
||||
}
|
||||
false // Done async
|
||||
}
|
||||
|
||||
private def fireAndForget(message: CamelMessage, exchange: CamelExchangeAdapter) {
|
||||
try {
|
||||
actorFor(endpoint.path) ! message
|
||||
} catch {
|
||||
case e ⇒ exchange.setFailure(new FailureResult(e))
|
||||
}
|
||||
}
|
||||
private def fireAndForget(message: CamelMessage, exchange: CamelExchangeAdapter): Unit =
|
||||
try { actorFor(endpoint.path) ! message } catch { case NonFatal(e) ⇒ exchange.setFailure(new FailureResult(e)) }
|
||||
|
||||
private[this] def actorFor(path: ActorEndpointPath): ActorRef =
|
||||
path.findActorIn(camel.system) getOrElse (throw new ActorNotRegisteredException(path.actorPath))
|
||||
|
||||
private[this] def messageFor(exchange: CamelExchangeAdapter) =
|
||||
exchange.toRequestMessage(Map(CamelMessage.MessageExchangeId -> exchange.getExchangeId))
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* For internal use only. Converts Strings to [[akka.util.Duration]]s
|
||||
*/
|
||||
private[camel] object DurationTypeConverter extends TypeConverter {
|
||||
def convertTo[T](`type`: Class[T], value: AnyRef) = {
|
||||
Duration(value.toString).asInstanceOf[T]
|
||||
override def convertTo[T](`type`: Class[T], value: AnyRef): T = `type`.cast(try {
|
||||
val d = Duration(value.toString)
|
||||
if (`type`.isInstance(d)) d else null
|
||||
} catch {
|
||||
case NonFatal(_) ⇒ null
|
||||
})
|
||||
|
||||
def convertTo[T](`type`: Class[T], exchange: Exchange, value: AnyRef): T = convertTo(`type`, value)
|
||||
def mandatoryConvertTo[T](`type`: Class[T], value: AnyRef): T = convertTo(`type`, value) match {
|
||||
case null ⇒ throw new NoTypeConversionAvailableException(value, `type`)
|
||||
case some ⇒ some
|
||||
}
|
||||
def convertTo[T](`type`: Class[T], exchange: Exchange, value: AnyRef) = convertTo(`type`, value)
|
||||
def mandatoryConvertTo[T](`type`: Class[T], value: AnyRef) = convertTo(`type`, value)
|
||||
def mandatoryConvertTo[T](`type`: Class[T], exchange: Exchange, value: AnyRef) = convertTo(`type`, value)
|
||||
def toString(duration: Duration) = duration.toNanos + " nanos"
|
||||
def mandatoryConvertTo[T](`type`: Class[T], exchange: Exchange, value: AnyRef): T = mandatoryConvertTo(`type`, value)
|
||||
def toString(duration: Duration): String = duration.toNanos + " nanos"
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -243,15 +242,15 @@ private[camel] case class ActorEndpointPath private (actorPath: String) {
|
|||
* For internal use only. Companion of `ActorEndpointPath`
|
||||
*/
|
||||
private[camel] object ActorEndpointPath {
|
||||
private val consumerConfig = new ConsumerConfig {}
|
||||
private val consumerConfig: ConsumerConfig = new ConsumerConfig {}
|
||||
|
||||
def apply(actorRef: ActorRef) = new ActorEndpointPath(actorRef.path.toString)
|
||||
def apply(actorRef: ActorRef): ActorEndpointPath = new ActorEndpointPath(actorRef.path.toString)
|
||||
|
||||
/**
|
||||
* Creates an [[akka.camel.internal.component.ActorEndpointPath]] from the remaining part of the endpoint URI (the part after the scheme, without the parameters of the URI).
|
||||
* Expects the remaining part of the URI (the actor path) in a format: path:%s
|
||||
*/
|
||||
def fromCamelPath(camelPath: String) = camelPath match {
|
||||
def fromCamelPath(camelPath: String): ActorEndpointPath = camelPath match {
|
||||
case id if id startsWith "path:" ⇒ new ActorEndpointPath(id substring 5)
|
||||
case _ ⇒ throw new IllegalArgumentException("Invalid path: [%s] - should be path:<actorPath>" format camelPath)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -13,7 +13,7 @@ import org.apache.camel.{ ProducerTemplate, CamelContext }
|
|||
* class is meant to be used from Java.
|
||||
*/
|
||||
abstract class UntypedConsumerActor extends UntypedActor with Consumer {
|
||||
final def endpointUri = getEndpointUri
|
||||
final def endpointUri: String = getEndpointUri
|
||||
|
||||
/**
|
||||
* Returns the Camel endpoint URI to consume messages from.
|
||||
|
|
|
|||
Some files were not shown because too many files have changed in this diff Show more
Loading…
Add table
Add a link
Reference in a new issue