From e8f16952a682c615afaa2706f24b878cb5b60ba5 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Sat, 31 Dec 2011 09:13:09 -0700 Subject: [PATCH 001/152] Avoid allocating varargs array and stack builder --- akka-actor/src/main/scala/akka/dispatch/Future.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index fea97fbaf3..db406986b7 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -317,7 +317,8 @@ object Future { new Runnable { def run = try { - val taskStack = Stack[() ⇒ Unit](task) + val taskStack = Stack.empty[() ⇒ Unit] + taskStack push task _taskStack set Some(taskStack) while (taskStack.nonEmpty) { val next = taskStack.pop() From a1f9b81ce9ad45aa05ef88ee1a0362141736d702 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Sat, 31 Dec 2011 09:18:37 -0700 Subject: [PATCH 002/152] Update IO to new iteratee based api --- .../src/test/scala/akka/actor/IOActor.scala | 389 ++++----- akka-actor/src/main/scala/akka/actor/IO.scala | 771 ++++++++++++------ .../src/main/scala/akka/util/ByteString.scala | 189 ++++- 3 files changed, 870 insertions(+), 479 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala index 757acb1fd0..06d3235409 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala @@ -4,8 +4,6 @@ package akka.actor -import org.scalatest.BeforeAndAfterEach - import akka.util.ByteString import akka.util.cps._ import scala.util.continuations._ @@ -13,214 +11,237 @@ import akka.testkit._ import akka.dispatch.{ Await, Future } object IOActorSpec { - import IO._ - class SimpleEchoServer(host: String, port: Int, ioManager: ActorRef, started: TestLatch) extends Actor { + class SimpleEchoServer(host: String, port: Int, started: TestLatch) extends Actor { - import context.dispatcher - implicit val timeout = context.system.settings.ActorTimeout + IO listen (host, port) - override def preStart = { - listen(ioManager, host, port) - started.open() - } + started.open - def createWorker = context.actorOf(Props(new Actor with IO { - def receiveIO = { - case NewClient(server) ⇒ - val socket = server.accept() - loopC { - val bytes = socket.read() - socket write bytes - } - } - })) + val state = IO.IterateeRef.Map.sync[IO.Handle]() def receive = { - case msg: NewClient ⇒ - createWorker forward msg - } - } - - class SimpleEchoClient(host: String, port: Int, ioManager: ActorRef) extends Actor with IO { - - lazy val socket: SocketHandle = connect(ioManager, host, port)(reader) - lazy val reader: ActorRef = context.actorOf(Props({ - new Actor with IO { - def receiveIO = { - case length: Int ⇒ - val bytes = socket.read(length) - sender ! bytes - } - } - })) - - def receiveIO = { - case bytes: ByteString ⇒ - socket write bytes - reader forward bytes.length - } - } - - // Basic Redis-style protocol - class KVStore(host: String, port: Int, ioManager: ActorRef, started: TestLatch) extends Actor { - - import context.dispatcher - implicit val timeout = context.system.settings.ActorTimeout - - var kvs: Map[String, ByteString] = Map.empty - - override def preStart = { - listen(ioManager, host, port) - started.open() - } - - def createWorker = context.actorOf(Props(new Actor with IO { - def receiveIO = { - case NewClient(server) ⇒ - val socket = server.accept() - loopC { - val cmd = socket.read(ByteString("\r\n")).utf8String - val result = matchC(cmd.split(' ')) { - case Array("SET", key, length) ⇒ - val value = socket read length.toInt - server.owner ? (('set, key, value)) map ((x: Any) ⇒ ByteString("+OK\r\n")) - case Array("GET", key) ⇒ - server.owner ? (('get, key)) map { - case Some(b: ByteString) ⇒ ByteString("$" + b.length + "\r\n") ++ b - case None ⇒ ByteString("$-1\r\n") - } - case Array("GETALL") ⇒ - server.owner ? 'getall map { - case m: Map[_, _] ⇒ - (ByteString("*" + (m.size * 2) + "\r\n") /: m) { - case (result, (k: String, v: ByteString)) ⇒ - val kBytes = ByteString(k) - result ++ ByteString("$" + kBytes.length + "\r\n") ++ kBytes ++ ByteString("$" + v.length + "\r\n") ++ v - } - } - } - result recover { - case e ⇒ ByteString("-" + e.getClass.toString + "\r\n") - } foreach { bytes ⇒ + case IO.NewClient(server) ⇒ + val socket = server.accept() + state(socket) flatMap { _ ⇒ + IO repeat { + IO.takeAny map { bytes ⇒ socket write bytes } } - } - })) + } + + case IO.Read(socket, bytes) ⇒ + state(socket)(IO Chunk bytes) + + case IO.Closed(socket, cause) ⇒ + state -= socket + + } + + } + + class SimpleEchoClient(host: String, port: Int) extends Actor { + + val socket = IO connect (host, port) + + val state = IO.IterateeRef.sync() def receive = { - case msg: NewClient ⇒ createWorker forward msg - case ('set, key: String, value: ByteString) ⇒ - kvs += (key -> value) - sender.tell((), self) - case ('get, key: String) ⇒ sender.tell(kvs.get(key), self) - case 'getall ⇒ sender.tell(kvs, self) + + case bytes: ByteString ⇒ + val source = sender + socket write bytes + for { + _ ← state + bytes ← IO take bytes.length + } yield source ! bytes + + case IO.Read(socket, bytes) ⇒ + state(IO Chunk bytes) + + case IO.Connected(socket) ⇒ + + case IO.Closed(socket, cause) ⇒ + + } + } + + sealed trait KVCommand { + def bytes: ByteString + } + + case class KVSet(key: String, value: String) extends KVCommand { + val bytes = ByteString("SET " + key + " " + value.length + "\r\n" + value + "\r\n") + } + + case class KVGet(key: String) extends KVCommand { + val bytes = ByteString("GET " + key + "\r\n") + } + + case object KVGetAll extends KVCommand { + val bytes = ByteString("GETALL\r\n") + } + + // Basic Redis-style protocol + class KVStore(host: String, port: Int, started: TestLatch) extends Actor { + + val state = IO.IterateeRef.Map.sync[IO.Handle]() + + var kvs: Map[String, String] = Map.empty + + IO listen (host, port) + + started.open + + val EOL = ByteString("\r\n") + + def receive = { + + case IO.NewClient(server) ⇒ + val socket = server.accept() + state(socket) flatMap { _ ⇒ + IO repeat { + IO takeUntil EOL map (_.utf8String split ' ') flatMap { + + case Array("SET", key, length) ⇒ + for { + value ← IO take length.toInt + _ ← IO takeUntil EOL + } yield { + kvs += (key -> value.utf8String) + ByteString("+OK\r\n") + } + + case Array("GET", key) ⇒ + IO Iteratee { + kvs get key map { value ⇒ + ByteString("$" + value.length + "\r\n" + value + "\r\n") + } getOrElse ByteString("$-1\r\n") + } + + case Array("GETALL") ⇒ + IO Iteratee { + (ByteString("*" + (kvs.size * 2) + "\r\n") /: kvs) { + case (result, (k, v)) ⇒ + val kBytes = ByteString(k) + val vBytes = ByteString(v) + result ++ + ByteString("$" + kBytes.length) ++ EOL ++ + kBytes ++ EOL ++ + ByteString("$" + vBytes.length) ++ EOL ++ + vBytes ++ EOL + } + } + + } map (socket write) + } + } + + case IO.Read(socket, bytes) ⇒ + state(socket)(IO Chunk bytes) + + case IO.Closed(socket, cause) ⇒ + state -= socket + } } - class KVClient(host: String, port: Int, ioManager: ActorRef) extends Actor with IO { + class KVClient(host: String, port: Int) extends Actor { - import context.dispatcher - implicit val timeout = context.system.settings.ActorTimeout + val socket = IO connect (host, port) - var socket: SocketHandle = _ + val state = IO.IterateeRef.sync() + + val EOL = ByteString("\r\n") + + def receive = { + case cmd: KVCommand ⇒ + val source = sender + socket write cmd.bytes + for { + _ ← state + result ← readResult + } yield result.fold(err ⇒ source ! Status.Failure(new RuntimeException(err)), source !) + + case IO.Read(socket, bytes) ⇒ + state(IO Chunk bytes) + + case IO.Connected(socket) ⇒ + + case IO.Closed(socket, cause) ⇒ - override def preStart { - socket = connect(ioManager, host, port) } - def reply(msg: Any) = sender.tell(msg, self) - - def receiveIO = { - case ('set, key: String, value: ByteString) ⇒ - socket write (ByteString("SET " + key + " " + value.length + "\r\n") ++ value) - reply(readResult) - - case ('get, key: String) ⇒ - socket write ByteString("GET " + key + "\r\n") - reply(readResult) - - case 'getall ⇒ - socket write ByteString("GETALL\r\n") - reply(readResult) - } - - def readResult = { - val resultType = socket.read(1).utf8String - resultType match { - case "+" ⇒ socket.read(ByteString("\r\n")).utf8String - case "-" ⇒ sys error socket.read(ByteString("\r\n")).utf8String + def readResult: IO.Iteratee[Either[String, Any]] = { + IO take 1 map (_.utf8String) flatMap { + case "+" ⇒ IO takeUntil EOL map (msg ⇒ Right(msg.utf8String)) + case "-" ⇒ IO takeUntil EOL map (err ⇒ Left(err.utf8String)) case "$" ⇒ - val length = socket.read(ByteString("\r\n")).utf8String - socket.read(length.toInt) - case "*" ⇒ - val count = socket.read(ByteString("\r\n")).utf8String - var result: Map[String, ByteString] = Map.empty - repeatC(count.toInt / 2) { - val k = readBytes - val v = readBytes - result += (k.utf8String -> v) + IO takeUntil EOL map (_.utf8String.toInt) flatMap { + case -1 ⇒ IO Iteratee Right(None) + case length ⇒ + for { + value ← IO take length + _ ← IO takeUntil EOL + } yield Right(Some(value.utf8String)) } - result - case _ ⇒ sys error "Unexpected response" + case "*" ⇒ + IO takeUntil EOL map (_.utf8String.toInt) flatMap { + case -1 ⇒ IO Iteratee Right(None) + case length ⇒ + IO.takeList(length)(readResult) map { list ⇒ + ((Right(Map()): Either[String, Map[String, String]]) /: list.grouped(2)) { + case (Right(m), List(Right(Some(k: String)), Right(Some(v: String)))) ⇒ Right(m + (k -> v)) + case (Right(_), _) ⇒ Left("Unexpected Response") + case (left, _) ⇒ left + } + } + } + case _ ⇒ IO Iteratee Left("Unexpected Response") } } - - def readBytes = { - val resultType = socket.read(1).utf8String - if (resultType != "$") sys error "Unexpected response" - val length = socket.read(ByteString("\r\n")).utf8String - socket.read(length.toInt) - } } - } @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) -class IOActorSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout { +class IOActorSpec extends AkkaSpec with DefaultTimeout { import IOActorSpec._ "an IO Actor" must { "run echo server" in { val started = TestLatch(1) - val ioManager = system.actorOf(Props(new IOManager(2))) // teeny tiny buffer - val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8064, ioManager, started))) - Await.ready(started, timeout.duration) - val client = system.actorOf(Props(new SimpleEchoClient("localhost", 8064, ioManager))) + val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8064, started))) + Await.ready(started, TestLatch.DefaultTimeout) + val client = system.actorOf(Props(new SimpleEchoClient("localhost", 8064))) val f1 = client ? ByteString("Hello World!1") val f2 = client ? ByteString("Hello World!2") val f3 = client ? ByteString("Hello World!3") - Await.result(f1, timeout.duration) must equal(ByteString("Hello World!1")) - Await.result(f2, timeout.duration) must equal(ByteString("Hello World!2")) - Await.result(f3, timeout.duration) must equal(ByteString("Hello World!3")) - system.stop(client) - system.stop(server) - system.stop(ioManager) + Await.result(f1, TestLatch.DefaultTimeout) must equal(ByteString("Hello World!1")) + Await.result(f2, TestLatch.DefaultTimeout) must equal(ByteString("Hello World!2")) + Await.result(f3, TestLatch.DefaultTimeout) must equal(ByteString("Hello World!3")) } "run echo server under high load" in { val started = TestLatch(1) - val ioManager = system.actorOf(Props(new IOManager())) - val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8065, ioManager, started))) - Await.ready(started, timeout.duration) - val client = system.actorOf(Props(new SimpleEchoClient("localhost", 8065, ioManager))) + val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8065, started))) + Await.ready(started, TestLatch.DefaultTimeout) + val client = system.actorOf(Props(new SimpleEchoClient("localhost", 8065))) val list = List.range(0, 1000) val f = Future.traverse(list)(i ⇒ client ? ByteString(i.toString)) - assert(Await.result(f, timeout.duration).size === 1000) - system.stop(client) - system.stop(server) - system.stop(ioManager) + assert(Await.result(f, TestLatch.DefaultTimeout).size === 1000) } + // Not currently configurable at runtime + /* "run echo server under high load with small buffer" in { val started = TestLatch(1) - val ioManager = system.actorOf(Props(new IOManager(2))) - val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8066, ioManager, started))) - Await.ready(started, timeout.duration) - val client = system.actorOf(Props(new SimpleEchoClient("localhost", 8066, ioManager))) + val ioManager = actorOf(new IOManager(2)) + val server = actorOf(new SimpleEchoServer("localhost", 8066, ioManager, started)) + started.await + val client = actorOf(new SimpleEchoClient("localhost", 8066, ioManager)) val list = List.range(0, 1000) val f = Future.traverse(list)(i ⇒ client ? ByteString(i.toString)) assert(Await.result(f, timeout.duration).size === 1000) @@ -228,32 +249,28 @@ class IOActorSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout { system.stop(server) system.stop(ioManager) } + */ "run key-value store" in { val started = TestLatch(1) - val ioManager = system.actorOf(Props(new IOManager(2))) // teeny tiny buffer - val server = system.actorOf(Props(new KVStore("localhost", 8067, ioManager, started))) - Await.ready(started, timeout.duration) - val client1 = system.actorOf(Props(new KVClient("localhost", 8067, ioManager))) - val client2 = system.actorOf(Props(new KVClient("localhost", 8067, ioManager))) - val f1 = client1 ? (('set, "hello", ByteString("World"))) - val f2 = client1 ? (('set, "test", ByteString("No one will read me"))) - val f3 = client1 ? (('get, "hello")) - Await.ready(f2, timeout.duration) - val f4 = client2 ? (('set, "test", ByteString("I'm a test!"))) - Await.ready(f4, timeout.duration) - val f5 = client1 ? (('get, "test")) - val f6 = client2 ? 'getall - Await.result(f1, timeout.duration) must equal("OK") - Await.result(f2, timeout.duration) must equal("OK") - Await.result(f3, timeout.duration) must equal(ByteString("World")) - Await.result(f4, timeout.duration) must equal("OK") - Await.result(f5, timeout.duration) must equal(ByteString("I'm a test!")) - Await.result(f6, timeout.duration) must equal(Map("hello" -> ByteString("World"), "test" -> ByteString("I'm a test!"))) - system.stop(client1) - system.stop(client2) - system.stop(server) - system.stop(ioManager) + val server = system.actorOf(Props(new KVStore("localhost", 8067, started))) + Await.ready(started, TestLatch.DefaultTimeout) + val client1 = system.actorOf(Props(new KVClient("localhost", 8067))) + val client2 = system.actorOf(Props(new KVClient("localhost", 8067))) + val f1 = client1 ? KVSet("hello", "World") + val f2 = client1 ? KVSet("test", "No one will read me") + val f3 = client1 ? KVGet("hello") + Await.ready(f2, TestLatch.DefaultTimeout) + val f4 = client2 ? KVSet("test", "I'm a test!") + Await.ready(f4, TestLatch.DefaultTimeout) + val f5 = client1 ? KVGet("test") + val f6 = client2 ? KVGetAll + Await.result(f1, TestLatch.DefaultTimeout) must equal("OK") + Await.result(f2, TestLatch.DefaultTimeout) must equal("OK") + Await.result(f3, TestLatch.DefaultTimeout) must equal(Some("World")) + Await.result(f4, TestLatch.DefaultTimeout) must equal("OK") + Await.result(f5, TestLatch.DefaultTimeout) must equal(Some("I'm a test!")) + Await.result(f6, TestLatch.DefaultTimeout) must equal(Map("hello" -> "World", "test" -> "I'm a test!")) } } diff --git a/akka-actor/src/main/scala/akka/actor/IO.scala b/akka-actor/src/main/scala/akka/actor/IO.scala index 28bad4f85e..89922c94ca 100644 --- a/akka-actor/src/main/scala/akka/actor/IO.scala +++ b/akka-actor/src/main/scala/akka/actor/IO.scala @@ -4,10 +4,8 @@ package akka.actor import akka.util.ByteString -import akka.dispatch.Envelope import java.net.InetSocketAddress import java.io.IOException -import java.util.concurrent.atomic.AtomicReference import java.nio.ByteBuffer import java.nio.channels.{ SelectableChannel, @@ -20,9 +18,8 @@ import java.nio.channels.{ CancelledKeyException } import scala.collection.mutable -import scala.collection.immutable.Queue import scala.annotation.tailrec -import scala.util.continuations._ +import scala.collection.generic.CanBuildFrom import com.eaio.uuid.UUID object IO { @@ -44,18 +41,6 @@ object IO { sealed trait ReadHandle extends Handle with Product { override def asReadable = this - - def read(len: Int)(implicit actor: Actor with IO): ByteString @cps[IOSuspendable[Any]] = shift { cont: (ByteString ⇒ IOSuspendable[Any]) ⇒ - ByteStringLength(cont, this, actor.context.asInstanceOf[ActorCell].currentMessage, len) - } - - def read()(implicit actor: Actor with IO): ByteString @cps[IOSuspendable[Any]] = shift { cont: (ByteString ⇒ IOSuspendable[Any]) ⇒ - ByteStringAny(cont, this, actor.context.asInstanceOf[ActorCell].currentMessage) - } - - def read(delimiter: ByteString, inclusive: Boolean = false)(implicit actor: Actor with IO): ByteString @cps[IOSuspendable[Any]] = shift { cont: (ByteString ⇒ IOSuspendable[Any]) ⇒ - ByteStringDelimited(cont, this, actor.context.asInstanceOf[ActorCell].currentMessage, delimiter, inclusive, 0) - } } sealed trait WriteHandle extends Handle with Product { @@ -89,259 +74,542 @@ object IO { case class Read(handle: ReadHandle, bytes: ByteString) extends IOMessage case class Write(handle: WriteHandle, bytes: ByteString) extends IOMessage - def listen(ioManager: ActorRef, address: InetSocketAddress)(implicit owner: ActorRef): ServerHandle = { + def listen(address: InetSocketAddress)(implicit context: ActorContext, owner: ActorRef): ServerHandle = { + val ioManager = IOManager.start()(context.system) val server = ServerHandle(owner, ioManager) ioManager ! Listen(server, address) server } - def listen(ioManager: ActorRef, host: String, port: Int)(implicit owner: ActorRef): ServerHandle = - listen(ioManager, new InetSocketAddress(host, port)) + def listen(host: String, port: Int)(implicit context: ActorContext, owner: ActorRef): ServerHandle = + listen(new InetSocketAddress(host, port))(context, owner) - def connect(ioManager: ActorRef, address: InetSocketAddress)(implicit owner: ActorRef): SocketHandle = { + def listen(address: InetSocketAddress, owner: ActorRef)(implicit context: ActorContext): ServerHandle = + listen(address)(context, owner) + + def listen(host: String, port: Int, owner: ActorRef)(implicit context: ActorContext): ServerHandle = + listen(new InetSocketAddress(host, port))(context, owner) + + def connect(address: InetSocketAddress)(implicit context: ActorContext, owner: ActorRef): SocketHandle = { + val ioManager = IOManager.start()(context.system) val socket = SocketHandle(owner, ioManager) ioManager ! Connect(socket, address) socket } - def connect(ioManager: ActorRef, host: String, port: Int)(implicit sender: ActorRef): SocketHandle = - connect(ioManager, new InetSocketAddress(host, port)) + def connect(host: String, port: Int)(implicit context: ActorContext, owner: ActorRef): SocketHandle = + connect(new InetSocketAddress(host, port))(context, owner) - private class HandleState(var readBytes: ByteString, var connected: Boolean) { - def this() = this(ByteString.empty, false) + def connect(address: InetSocketAddress, owner: ActorRef)(implicit context: ActorContext): SocketHandle = + connect(address)(context, owner) + + def connect(host: String, port: Int, owner: ActorRef)(implicit context: ActorContext): SocketHandle = + connect(new InetSocketAddress(host, port))(context, owner) + + sealed trait Input { + def ++(that: Input): Input } - sealed trait IOSuspendable[+A] - sealed trait CurrentMessage { def message: Envelope } - private case class ByteStringLength(continuation: (ByteString) ⇒ IOSuspendable[Any], handle: Handle, message: Envelope, length: Int) extends IOSuspendable[ByteString] with CurrentMessage - private case class ByteStringDelimited(continuation: (ByteString) ⇒ IOSuspendable[Any], handle: Handle, message: Envelope, delimter: ByteString, inclusive: Boolean, scanned: Int) extends IOSuspendable[ByteString] with CurrentMessage - private case class ByteStringAny(continuation: (ByteString) ⇒ IOSuspendable[Any], handle: Handle, message: Envelope) extends IOSuspendable[ByteString] with CurrentMessage - private case class Retry(message: Envelope) extends IOSuspendable[Nothing] - private case object Idle extends IOSuspendable[Nothing] - -} - -trait IO { - this: Actor ⇒ - import IO._ - - type ReceiveIO = PartialFunction[Any, Any @cps[IOSuspendable[Any]]] - - implicit protected def ioActor: Actor with IO = this - - private val _messages: mutable.Queue[Envelope] = mutable.Queue.empty - - private var _state: Map[Handle, HandleState] = Map.empty - - private var _next: IOSuspendable[Any] = Idle - - private def state(handle: Handle): HandleState = _state.get(handle) match { - case Some(s) ⇒ s - case _ ⇒ - val s = new HandleState() - _state += (handle -> s) - s + object Chunk { + val empty = Chunk(ByteString.empty) } - final def receive: Receive = { - case Read(handle, newBytes) ⇒ - val st = state(handle) - st.readBytes ++= newBytes - run() - case Connected(socket) ⇒ - state(socket).connected = true - run() - case msg @ Closed(handle, _) ⇒ - _state -= handle // TODO: clean up better - if (_receiveIO.isDefinedAt(msg)) { - _next = reset { _receiveIO(msg); Idle } - } - run() - case msg if _next ne Idle ⇒ - _messages enqueue context.asInstanceOf[ActorCell].currentMessage - case msg if _receiveIO.isDefinedAt(msg) ⇒ - _next = reset { _receiveIO(msg); Idle } - run() - } - - def receiveIO: ReceiveIO - - def retry(): Any @cps[IOSuspendable[Any]] = - shift { _: (Any ⇒ IOSuspendable[Any]) ⇒ - _next match { - case n: CurrentMessage ⇒ Retry(n.message) - case _ ⇒ Idle - } - } - - private lazy val _receiveIO = receiveIO - - // only reinvoke messages from the original message to avoid stack overflow - private var reinvoked = false - private def reinvoke() { - if (!reinvoked && (_next eq Idle) && _messages.nonEmpty) { - try { - reinvoked = true - while ((_next eq Idle) && _messages.nonEmpty) self.asInstanceOf[LocalActorRef].underlying invoke _messages.dequeue - } finally { - reinvoked = false - } + case class Chunk(bytes: ByteString) extends Input { + def ++(that: Input) = that match { + case Chunk(more) ⇒ Chunk(bytes ++ more) + case _: EOF ⇒ that } } - @tailrec - private def run() { - _next match { - case ByteStringLength(continuation, handle, message, waitingFor) ⇒ - context.asInstanceOf[ActorCell].currentMessage = message - val st = state(handle) - if (st.readBytes.length >= waitingFor) { - val bytes = st.readBytes.take(waitingFor) //.compact - st.readBytes = st.readBytes.drop(waitingFor) - _next = continuation(bytes) - run() - } - case bsd @ ByteStringDelimited(continuation, handle, message, delimiter, inclusive, scanned) ⇒ - context.asInstanceOf[ActorCell].currentMessage = message - val st = state(handle) - val idx = st.readBytes.indexOfSlice(delimiter, scanned) - if (idx >= 0) { - val index = if (inclusive) idx + delimiter.length else idx - val bytes = st.readBytes.take(index) //.compact - st.readBytes = st.readBytes.drop(idx + delimiter.length) - _next = continuation(bytes) - run() + case class EOF(cause: Option[Exception]) extends Input { + def ++(that: Input) = this + } + + object Iteratee { + def apply[A](value: A): Iteratee[A] = Done(value) + def apply(): Iteratee[Unit] = unit + val unit: Iteratee[Unit] = Done(()) + } + + /** + * A basic Iteratee implementation of Oleg's Iteratee (http://okmij.org/ftp/Streams.html). + * No support for Enumerator or Input types other then ByteString at the moment. + */ + sealed abstract class Iteratee[+A] { + + /** + * Applies the given input to the Iteratee, returning the resulting Iteratee + * and the unused Input. + */ + final def apply(input: Input): (Iteratee[A], Input) = this match { + case Cont(f) ⇒ f(input) + case iter ⇒ (iter, input) + } + + final def get: A = this(EOF(None))._1 match { + case Done(value) ⇒ value + case Cont(_) ⇒ sys.error("Divergent Iteratee") + case Failure(e) ⇒ throw e + } + + final def flatMap[B](f: A ⇒ Iteratee[B]): Iteratee[B] = this match { + case Done(value) ⇒ f(value) + case Cont(k: Chain[_]) ⇒ Cont(k :+ f) + case Cont(k) ⇒ Cont(Chain(k, f)) + case failure: Failure ⇒ failure + } + + final def map[B](f: A ⇒ B): Iteratee[B] = this match { + case Done(value) ⇒ Done(f(value)) + case Cont(k: Chain[_]) ⇒ Cont(k :+ ((a: A) ⇒ Done(f(a)))) + case Cont(k) ⇒ Cont(Chain(k, (a: A) ⇒ Done(f(a)))) + case failure: Failure ⇒ failure + } + + } + + /** + * An Iteratee representing a result and the remaining ByteString. Also used to + * wrap any constants or precalculated values that need to be composed with + * other Iteratees. + */ + final case class Done[+A](result: A) extends Iteratee[A] + + /** + * An Iteratee that still requires more input to calculate it's result. + */ + final case class Cont[+A](f: Input ⇒ (Iteratee[A], Input)) extends Iteratee[A] + + /** + * An Iteratee representing a failure to calcualte a result. + * FIXME: move into 'Cont' as in Oleg's implementation + */ + final case class Failure(exception: Throwable) extends Iteratee[Nothing] + + object IterateeRef { + def sync[A](initial: Iteratee[A]): IterateeRefSync[A] = new IterateeRefSync(initial) + def sync(): IterateeRefSync[Unit] = new IterateeRefSync(Iteratee.unit) + + def async[A](initial: Iteratee[A])(implicit app: ActorSystem): IterateeRefAsync[A] = new IterateeRefAsync(initial) + def async()(implicit app: ActorSystem): IterateeRefAsync[Unit] = new IterateeRefAsync(Iteratee.unit) + + 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 empty = new Map[K, V](refFactory) + } + object Map { + def apply[K, V](refFactory: ⇒ IterateeRef[V]): IterateeRef.Map[K, V] = new Map(refFactory) + def sync[K](): IterateeRef.Map[K, Unit] = new Map(IterateeRef.sync()) + def async[K]()(implicit app: ActorSystem): IterateeRef.Map[K, Unit] = new Map(IterateeRef.async()) + } + } + + /** + * A mutable reference to an Iteratee. Not thread safe. + * + * Designed for use within an Actor. + * + * Includes mutable implementations of flatMap, map, and apply which + * update the internal reference and return Unit. + */ + trait IterateeRef[A] { + def flatMap(f: A ⇒ Iteratee[A]): Unit + def map(f: A ⇒ A): Unit + def apply(input: Input): Unit + } + + 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 { + 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) + def value: (Iteratee[A], Input) = _value + } + + final class IterateeRefAsync[A](initial: Iteratee[A])(implicit app: ActorSystem) extends IterateeRef[A] { + import akka.dispatch.Future + private var _value: Future[(Iteratee[A], Input)] = Future((initial, Chunk.empty)) + 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)) + def future: Future[(Iteratee[A], Input)] = _value + } + + /** + * An Iteratee that returns the ByteString prefix up until the supplied delimiter. + * The delimiter is dropped by default, but it can be returned with the result by + * setting 'inclusive' to be 'true'. + */ + def takeUntil(delimiter: ByteString, inclusive: Boolean = false): Iteratee[ByteString] = { + def step(taken: ByteString)(input: Input): (Iteratee[ByteString], Input) = input match { + case Chunk(more) ⇒ + val bytes = taken ++ more + val startIdx = bytes.indexOfSlice(delimiter, math.max(taken.length - delimiter.length, 0)) + if (startIdx >= 0) { + val endIdx = startIdx + delimiter.length + (Done(bytes take (if (inclusive) endIdx else startIdx)), Chunk(bytes drop endIdx)) } else { - _next = bsd.copy(scanned = math.min(idx - delimiter.length, 0)) + (Cont(step(bytes)), Chunk.empty) } - case ByteStringAny(continuation, handle, message) ⇒ - context.asInstanceOf[ActorCell].currentMessage = message - val st = state(handle) - if (st.readBytes.length > 0) { - val bytes = st.readBytes //.compact - st.readBytes = ByteString.empty - _next = continuation(bytes) - run() + case eof ⇒ (Cont(step(taken)), eof) + } + + Cont(step(ByteString.empty)) + } + + def takeWhile(p: (Byte) ⇒ Boolean): Iteratee[ByteString] = { + def step(taken: ByteString)(input: Input): (Iteratee[ByteString], Input) = input match { + case Chunk(more) ⇒ + val (found, rest) = more span p + if (rest.isEmpty) + (Cont(step(taken ++ found)), Chunk.empty) + else + (Done(taken ++ found), Chunk(rest)) + case eof ⇒ (Done(taken), eof) + } + + Cont(step(ByteString.empty)) + } + + /** + * An Iteratee that returns a ByteString of the requested length. + */ + def take(length: Int): Iteratee[ByteString] = { + def step(taken: ByteString)(input: Input): (Iteratee[ByteString], Input) = input match { + case Chunk(more) ⇒ + val bytes = taken ++ more + if (bytes.length >= length) + (Done(bytes.take(length)), Chunk(bytes.drop(length))) + else + (Cont(step(bytes)), Chunk.empty) + case eof ⇒ (Cont(step(taken)), eof) + } + + Cont(step(ByteString.empty)) + } + + /** + * An Iteratee that ignores the specified number of bytes. + */ + def drop(length: Int): Iteratee[Unit] = { + def step(left: Int)(input: Input): (Iteratee[Unit], Input) = input match { + case Chunk(more) ⇒ + if (left > more.length) + (Cont(step(left - more.length)), Chunk.empty) + else + (Done(), Chunk(more drop left)) + case eof ⇒ (Done(), eof) + } + + Cont(step(length)) + } + + /** + * An Iteratee that returns the remaining ByteString until an EOF is given. + */ + val takeAll: Iteratee[ByteString] = { + def step(taken: ByteString)(input: Input): (Iteratee[ByteString], Input) = input match { + case Chunk(more) ⇒ + val bytes = taken ++ more + (Cont(step(bytes)), Chunk.empty) + case eof ⇒ (Done(taken), eof) + } + + Cont(step(ByteString.empty)) + } + + /** + * An Iteratee that returns any input it receives + */ + val takeAny: Iteratee[ByteString] = Cont { + case Chunk(bytes) if bytes.nonEmpty ⇒ (Done(bytes), Chunk.empty) + case Chunk(bytes) ⇒ (takeAny, Chunk.empty) + case eof ⇒ (Done(ByteString.empty), eof) + } + + def takeList[A](length: Int)(iter: Iteratee[A]): Iteratee[List[A]] = { + def step(left: Int, list: List[A]): Iteratee[List[A]] = + if (left == 0) Done(list.reverse) + else iter flatMap (a ⇒ step(left - 1, a :: list)) + + step(length, Nil) + } + + def peek(length: Int): Iteratee[ByteString] = { + def step(taken: ByteString)(input: Input): (Iteratee[ByteString], Input) = input match { + case Chunk(more) ⇒ + val bytes = taken ++ more + if (bytes.length >= length) + (Done(bytes.take(length)), Chunk(bytes)) + else + (Cont(step(bytes)), Chunk.empty) + case eof ⇒ (Cont(step(taken)), eof) + } + + Cont(step(ByteString.empty)) + } + + def repeat(iter: Iteratee[Unit]): Iteratee[Unit] = + iter flatMap (_ ⇒ repeat(iter)) + + def traverse[A, B, M[A] <: Traversable[A]](in: M[A])(f: A ⇒ Iteratee[B])(implicit cbf: CanBuildFrom[M[A], B, M[B]]): Iteratee[M[B]] = + fold(cbf(in), in)((b, a) ⇒ f(a) map (b += _)) map (_.result) + + def fold[A, B, M[A] <: Traversable[A]](initial: B, in: M[A])(f: (B, A) ⇒ Iteratee[B]): Iteratee[B] = + (Iteratee(initial) /: in)((ib, a) ⇒ ib flatMap (b ⇒ f(b, a))) + + // private api + + private object Chain { + def apply[A](f: Input ⇒ (Iteratee[A], Input)) = new Chain[A](f, Nil, Nil) + def apply[A, B](f: Input ⇒ (Iteratee[A], Input), k: A ⇒ Iteratee[B]) = new Chain[B](f, List(k.asInstanceOf[Any ⇒ Iteratee[Any]]), Nil) + } + + /** + * A function 'ByteString => Iteratee[A]' that composes with 'A => Iteratee[B]' functions + * in a stack-friendly manner. + * + * For internal use within Iteratee. + */ + private final case class Chain[A] private (cur: Input ⇒ (Iteratee[Any], Input), queueOut: List[Any ⇒ Iteratee[Any]], queueIn: List[Any ⇒ Iteratee[Any]]) extends (Input ⇒ (Iteratee[A], Input)) { + + def :+[B](f: A ⇒ Iteratee[B]) = new Chain[B](cur, queueOut, f.asInstanceOf[Any ⇒ Iteratee[Any]] :: queueIn) + + def apply(input: Input): (Iteratee[A], Input) = { + @tailrec + def run(result: (Iteratee[Any], Input), queueOut: List[Any ⇒ Iteratee[Any]], queueIn: List[Any ⇒ Iteratee[Any]]): (Iteratee[Any], Input) = { + if (queueOut.isEmpty) { + if (queueIn.isEmpty) result + else run(result, queueIn.reverse, Nil) + } else result match { + case (Done(value), rest) ⇒ + queueOut.head(value) match { + //case Cont(Chain(f, q)) ⇒ run(f(rest), q ++ tail) <- can cause big slowdown, need to test if needed + case Cont(f) ⇒ run(f(rest), queueOut.tail, queueIn) + case iter ⇒ run((iter, rest), queueOut.tail, queueIn) + } + case (Cont(f), rest) ⇒ + (Cont(new Chain(f, queueOut, queueIn)), rest) + case _ ⇒ result } - case Retry(message) ⇒ - message +=: _messages - _next = Idle - run() - case Idle ⇒ reinvoke() + } + run(cur(input), queueOut, queueIn).asInstanceOf[(Iteratee[A], Input)] } } + } -class IOManager(bufferSize: Int = 8192) extends Actor { +object IOManager { + def start()(implicit system: ActorSystem): ActorRef = { + // TODO: Replace with better "get or create" if/when available + val ref = system.actorFor(system / "io-manager") + if (!ref.isInstanceOf[EmptyLocalActorRef]) ref else try { + system.actorOf(Props[IOManager], "io-manager") + } catch { + case _: InvalidActorNameException ⇒ ref + } + } + def stop()(implicit system: ActorSystem): Unit = { + // TODO: send shutdown message to IOManager + } + +} + +final class WriteBuffer(bufferSize: Int) { + private val _queue = new java.util.ArrayDeque[ByteString] + private val _buffer = ByteBuffer.allocate(bufferSize) + private var _length = 0 + + private def fillBuffer(): Boolean = { + while (!_queue.isEmpty && _buffer.hasRemaining) { + val next = _queue.pollFirst + val rest = next.drop(next.copyToBuffer(_buffer)) + if (rest.nonEmpty) _queue.offerFirst(rest) + } + !_buffer.hasRemaining + } + + def enqueue(elem: ByteString): this.type = { + _length += elem.length + val rest = elem.drop(elem.copyToBuffer(_buffer)) + if (rest.nonEmpty) _queue.offerLast(rest) + this + } + + def length = _length + + def isEmpty = _length == 0 + + def write(channel: WritableByteChannel with SelectableChannel): Int = { + @tailrec + def run(total: Int): Int = { + if (this.isEmpty) total + else { + val written = try { + _buffer.flip() + channel write _buffer + } finally { + // don't leave buffer in wrong state + _buffer.compact() + fillBuffer() + } + _length -= written + if (_buffer.position > 0) { + total + written + } else { + run(total + written) + } + } + } + + run(0) + } + +} + +// TODO: Support a pool of workers +final class IOManager extends Actor { import SelectionKey.{ OP_READ, OP_WRITE, OP_ACCEPT, OP_CONNECT } - import IOWorker._ - var worker: IOWorker = _ + val bufferSize = 8192 // TODO: make buffer size configurable - override def preStart { - worker = new IOWorker(context.system, self, bufferSize) - worker.start() + type ReadChannel = ReadableByteChannel with SelectableChannel + type WriteChannel = WritableByteChannel with SelectableChannel + + val selector: Selector = Selector open () + + val channels = mutable.Map.empty[IO.Handle, SelectableChannel] + + val accepted = mutable.Map.empty[IO.ServerHandle, mutable.Queue[SelectableChannel]] + + val writes = mutable.Map.empty[IO.WriteHandle, WriteBuffer] + + val closing = mutable.Set.empty[IO.Handle] + + val buffer = ByteBuffer.allocate(bufferSize) + + var lastSelect = 0 + + val selectAt = 100 // TODO: determine best value, perhaps based on throughput? Other triggers (like write queue size)? + + //val selectEveryNanos = 1000000 // nanos + + //var lastSelectNanos = System.nanoTime + + var running = false + + var selectSent = false + + var fastSelect = false + + object Select + + def run() { + if (!running) { + running = true + if (!selectSent) { + selectSent = true + self ! Select + } + } + lastSelect += 1 + if (lastSelect >= selectAt /* || (lastSelectNanos + selectEveryNanos) < System.nanoTime */ ) select() + } + + def select() { + if (selector.isOpen) { + // TODO: Make select behaviour configurable. + // Blocking 1ms reduces allocations during idle times, non blocking gives better performance. + if (fastSelect) selector.selectNow else selector.select(1) + val keys = selector.selectedKeys.iterator + fastSelect = keys.hasNext + while (keys.hasNext) { + val key = keys.next() + keys.remove() + if (key.isValid) { process(key) } + } + if (channels.isEmpty) running = false + } else { + running = false + } + //lastSelectNanos = System.nanoTime + lastSelect = 0 } def receive = { + case Select ⇒ + select() + if (running) self ! Select + selectSent = running + case IO.Listen(server, address) ⇒ val channel = ServerSocketChannel open () channel configureBlocking false - channel.socket bind address - worker(Register(server, channel, OP_ACCEPT)) + channel.socket bind (address, 1000) // TODO: make backlog configurable + channels update (server, channel) + channel register (selector, OP_ACCEPT, server) + run() case IO.Connect(socket, address) ⇒ val channel = SocketChannel open () channel configureBlocking false channel connect address - worker(Register(socket, channel, OP_CONNECT | OP_READ)) + channels update (socket, channel) + channel register (selector, OP_CONNECT | OP_READ, socket) + run() - case IO.Accept(socket, server) ⇒ worker(Accepted(socket, server)) - case IO.Write(handle, data) ⇒ worker(Write(handle, data.asByteBuffer)) - case IO.Close(handle) ⇒ worker(Close(handle)) + case IO.Accept(socket, server) ⇒ + val queue = accepted(server) + val channel = queue.dequeue() + channels update (socket, channel) + channel register (selector, OP_READ, socket) + run() + + case IO.Write(handle, data) ⇒ + if (channels contains handle) { + val queue = { + val existing = writes get handle + if (existing.isDefined) existing.get + else { + val q = new WriteBuffer(bufferSize) + writes update (handle, q) + q + } + } + if (queue.isEmpty) addOps(handle, OP_WRITE) + queue enqueue data + if (queue.length >= bufferSize) write(handle, channels(handle).asInstanceOf[WriteChannel]) + } + run() + + case IO.Close(handle: IO.WriteHandle) ⇒ + if (writes get handle filterNot (_.isEmpty) isDefined) { + closing += handle + } else { + cleanup(handle, None) + } + run() + + case IO.Close(handle) ⇒ + cleanup(handle, None) + run() } override def postStop { - worker(Shutdown) + channels.keys foreach (handle ⇒ cleanup(handle, None)) + selector.close } -} - -private[akka] object IOWorker { - sealed trait Request - case class Register(handle: IO.Handle, channel: SelectableChannel, ops: Int) extends Request - case class Accepted(socket: IO.SocketHandle, server: IO.ServerHandle) extends Request - case class Write(handle: IO.WriteHandle, data: ByteBuffer) extends Request - case class Close(handle: IO.Handle) extends Request - case object Shutdown extends Request -} - -private[akka] class IOWorker(system: ActorSystem, ioManager: ActorRef, val bufferSize: Int) { - import SelectionKey.{ OP_READ, OP_WRITE, OP_ACCEPT, OP_CONNECT } - import IOWorker._ - - type ReadChannel = ReadableByteChannel with SelectableChannel - type WriteChannel = WritableByteChannel with SelectableChannel - - implicit val optionIOManager: Some[ActorRef] = Some(ioManager) - - def apply(request: Request): Unit = - addRequest(request) - - def start(): Unit = - thread.start() - - // private - - private val selector: Selector = Selector open () - - private val _requests = new AtomicReference(List.empty[Request]) - - private var accepted = Map.empty[IO.ServerHandle, Queue[SelectableChannel]].withDefaultValue(Queue.empty) - - private var channels = Map.empty[IO.Handle, SelectableChannel] - - private var writes = Map.empty[IO.WriteHandle, Queue[ByteBuffer]].withDefaultValue(Queue.empty) - - private val buffer = ByteBuffer.allocate(bufferSize) - - private val thread = new Thread("io-worker") { - override def run() { - while (selector.isOpen) { - selector select () - val keys = selector.selectedKeys.iterator - while (keys.hasNext) { - val key = keys next () - keys remove () - if (key.isValid) { process(key) } - } - _requests.getAndSet(Nil).reverse foreach { - case Register(handle, channel, ops) ⇒ - channels += (handle -> channel) - channel register (selector, ops, handle) - case Accepted(socket, server) ⇒ - val (channel, rest) = accepted(server).dequeue - if (rest.isEmpty) accepted -= server - else accepted += (server -> rest) - channels += (socket -> channel) - channel register (selector, OP_READ, socket) - case Write(handle, data) ⇒ - if (channels contains handle) { - val queue = writes(handle) - if (queue.isEmpty) addOps(handle, OP_WRITE) - writes += (handle -> queue.enqueue(data)) - } - case Close(handle) ⇒ - cleanup(handle, None) - case Shutdown ⇒ - channels.values foreach (_.close) - selector.close - } - } - } - } - - private def process(key: SelectionKey) { + def process(key: SelectionKey) { val handle = key.attachment.asInstanceOf[IO.Handle] try { if (key.isConnectable) key.channel match { @@ -369,7 +637,8 @@ private[akka] class IOWorker(system: ActorSystem, ioManager: ActorRef, val buffe } } - private def cleanup(handle: IO.Handle, cause: Option[Exception]) { + def cleanup(handle: IO.Handle, cause: Option[Exception]) { + closing -= handle handle match { case server: IO.ServerHandle ⇒ accepted -= server case writable: IO.WriteHandle ⇒ writes -= writable @@ -378,28 +647,27 @@ private[akka] class IOWorker(system: ActorSystem, ioManager: ActorRef, val buffe case Some(channel) ⇒ channel.close channels -= handle - // TODO: what if handle.owner is no longer running? - handle.owner ! IO.Closed(handle, cause) + if (!handle.owner.isTerminated) handle.owner ! IO.Closed(handle, cause) case None ⇒ } } - private def setOps(handle: IO.Handle, ops: Int): Unit = + def setOps(handle: IO.Handle, ops: Int): Unit = channels(handle) keyFor selector interestOps ops - private def addOps(handle: IO.Handle, ops: Int) { + def addOps(handle: IO.Handle, ops: Int) { val key = channels(handle) keyFor selector val cur = key.interestOps key interestOps (cur | ops) } - private def removeOps(handle: IO.Handle, ops: Int) { + def removeOps(handle: IO.Handle, ops: Int) { val key = channels(handle) keyFor selector val cur = key.interestOps key interestOps (cur - (cur & ops)) } - private def connect(socket: IO.SocketHandle, channel: SocketChannel) { + def connect(socket: IO.SocketHandle, channel: SocketChannel) { if (channel.finishConnect) { removeOps(socket, OP_CONNECT) socket.owner ! IO.Connected(socket) @@ -409,18 +677,27 @@ private[akka] class IOWorker(system: ActorSystem, ioManager: ActorRef, val buffe } @tailrec - private def accept(server: IO.ServerHandle, channel: ServerSocketChannel) { + def accept(server: IO.ServerHandle, channel: ServerSocketChannel) { val socket = channel.accept if (socket ne null) { socket configureBlocking false - accepted += (server -> (accepted(server) enqueue socket)) + val queue = { + val existing = accepted get server + if (existing.isDefined) existing.get + else { + val q = mutable.Queue[SelectableChannel]() + accepted update (server, q) + q + } + } + queue += socket server.owner ! IO.NewClient(server) accept(server, channel) } } @tailrec - private def read(handle: IO.ReadHandle, channel: ReadChannel) { + def read(handle: IO.ReadHandle, channel: ReadChannel) { buffer.clear val readLen = channel read buffer if (readLen == -1) { @@ -432,30 +709,16 @@ private[akka] class IOWorker(system: ActorSystem, ioManager: ActorRef, val buffe } } - @tailrec - private def write(handle: IO.WriteHandle, channel: WriteChannel) { + def write(handle: IO.WriteHandle, channel: WriteChannel) { val queue = writes(handle) - if (queue.nonEmpty) { - val (buf, bufs) = queue.dequeue - val writeLen = channel write buf - if (buf.remaining == 0) { - if (bufs.isEmpty) { - writes -= handle - removeOps(handle, OP_WRITE) - } else { - writes += (handle -> bufs) - write(handle, channel) - } + queue write channel + if (queue.isEmpty) { + if (closing(handle)) { + cleanup(handle, None) + } else { + removeOps(handle, OP_WRITE) } } } - @tailrec - private def addRequest(req: Request) { - val requests = _requests.get - if (_requests compareAndSet (requests, req :: requests)) - selector wakeup () - else - addRequest(req) - } } diff --git a/akka-actor/src/main/scala/akka/util/ByteString.scala b/akka-actor/src/main/scala/akka/util/ByteString.scala index 948489c335..fb41df2429 100644 --- a/akka-actor/src/main/scala/akka/util/ByteString.scala +++ b/akka-actor/src/main/scala/akka/util/ByteString.scala @@ -1,10 +1,11 @@ package akka.util import java.nio.ByteBuffer +import java.nio.charset.Charset -import scala.collection.IndexedSeqOptimized -import scala.collection.mutable.{ Builder, ArrayBuilder } -import scala.collection.immutable.IndexedSeq +import scala.collection.{ IndexedSeqOptimized, LinearSeq } +import scala.collection.mutable.{ Builder, ArrayBuilder, WrappedArray } +import scala.collection.immutable.{ IndexedSeq, VectorBuilder } import scala.collection.generic.{ CanBuildFrom, GenericCompanion } object ByteString { @@ -30,35 +31,43 @@ object ByteString { def apply(string: String, charset: String): ByteString = ByteString1(string.getBytes(charset)) + def fromArray(array: Array[Byte], offset: Int, length: Int): ByteString = { + val copyOffset = math.max(offset, 0) + val copyLength = math.max(math.min(array.length - copyOffset, length), 0) + if (copyLength == 0) empty + else { + val copyArray = new Array[Byte](copyLength) + Array.copy(array, copyOffset, copyArray, 0, copyLength) + ByteString1(copyArray) + } + } + val empty: ByteString = ByteString1(Array.empty[Byte]) - def newBuilder: Builder[Byte, ByteString] = new ArrayBuilder.ofByte mapResult apply + def newBuilder = new ByteStringBuilder implicit def canBuildFrom = new CanBuildFrom[TraversableOnce[Byte], Byte, ByteString] { def apply(from: TraversableOnce[Byte]) = newBuilder def apply() = newBuilder } - private object ByteString1 { + private[akka] object ByteString1 { def apply(bytes: Array[Byte]) = new ByteString1(bytes) } - final class ByteString1 private (bytes: Array[Byte], startIndex: Int, endIndex: Int) extends ByteString { + final class ByteString1 private (private val bytes: Array[Byte], private val startIndex: Int, val length: Int) extends ByteString { private def this(bytes: Array[Byte]) = this(bytes, 0, bytes.length) def apply(idx: Int): Byte = bytes(checkRangeConvert(idx)) private def checkRangeConvert(index: Int) = { - val idx = index + startIndex - if (0 <= index && idx < endIndex) - idx + if (0 <= index && length > index) + index + startIndex else throw new IndexOutOfBoundsException(index.toString) } - def length: Int = endIndex - startIndex - def toArray: Array[Byte] = { val ar = new Array[Byte](length) Array.copy(bytes, startIndex, ar, 0, length) @@ -68,8 +77,7 @@ object ByteString { override def clone: ByteString = new ByteString1(toArray) def compact: ByteString = - if (startIndex == 0 && endIndex == bytes.length) this - else clone + if (length == bytes.length) this else clone def asByteBuffer: ByteBuffer = { val buffer = ByteBuffer.wrap(bytes, startIndex, length).asReadOnlyBuffer @@ -77,8 +85,8 @@ object ByteString { else buffer } - def utf8String: String = - new String(if (startIndex == 0 && endIndex == bytes.length) bytes else toArray, "UTF-8") + def decodeString(charset: String): String = + new String(if (length == bytes.length) bytes else toArray, charset) def ++(that: ByteString): ByteString = that match { case b: ByteString1 ⇒ ByteStrings(this, b) @@ -87,42 +95,50 @@ object ByteString { override def slice(from: Int, until: Int): ByteString = { val newStartIndex = math.max(from, 0) + startIndex - val newEndIndex = math.min(until, length) + startIndex - if (newEndIndex <= newStartIndex) ByteString.empty - else new ByteString1(bytes, newStartIndex, newEndIndex) + val newLength = math.min(until, length) - from + if (newLength <= 0) ByteString.empty + else new ByteString1(bytes, newStartIndex, newLength) } override def copyToArray[A >: Byte](xs: Array[A], start: Int, len: Int): Unit = Array.copy(bytes, startIndex, xs, start, math.min(math.min(length, len), xs.length - start)) + def copyToBuffer(buffer: ByteBuffer): Int = { + val copyLength = math.min(buffer.remaining, length) + if (copyLength > 0) buffer.put(bytes, startIndex, copyLength) + copyLength + } + } - private object ByteStrings { - def apply(bytestrings: Vector[ByteString1]): ByteString = new ByteStrings(bytestrings) + private[akka] object ByteStrings { + def apply(bytestrings: Vector[ByteString1]): ByteString = new ByteStrings(bytestrings, (0 /: bytestrings)(_ + _.length)) + + def apply(bytestrings: Vector[ByteString1], length: Int): ByteString = new ByteStrings(bytestrings, length) def apply(b1: ByteString1, b2: ByteString1): ByteString = compare(b1, b2) match { - case 3 ⇒ new ByteStrings(Vector(b1, b2)) + case 3 ⇒ new ByteStrings(Vector(b1, b2), b1.length + b2.length) case 2 ⇒ b2 case 1 ⇒ b1 case 0 ⇒ ByteString.empty } def apply(b: ByteString1, bs: ByteStrings): ByteString = compare(b, bs) match { - case 3 ⇒ new ByteStrings(b +: bs.bytestrings) + case 3 ⇒ new ByteStrings(b +: bs.bytestrings, bs.length + b.length) case 2 ⇒ bs case 1 ⇒ b case 0 ⇒ ByteString.empty } def apply(bs: ByteStrings, b: ByteString1): ByteString = compare(bs, b) match { - case 3 ⇒ new ByteStrings(bs.bytestrings :+ b) + case 3 ⇒ new ByteStrings(bs.bytestrings :+ b, bs.length + b.length) case 2 ⇒ b case 1 ⇒ bs case 0 ⇒ ByteString.empty } def apply(bs1: ByteStrings, bs2: ByteStrings): ByteString = compare(bs1, bs2) match { - case 3 ⇒ new ByteStrings(bs1.bytestrings ++ bs2.bytestrings) + case 3 ⇒ new ByteStrings(bs1.bytestrings ++ bs2.bytestrings, bs1.length + bs2.length) case 2 ⇒ bs2 case 1 ⇒ bs1 case 0 ⇒ ByteString.empty @@ -133,9 +149,10 @@ object ByteString { if (b1.length == 0) if (b2.length == 0) 0 else 2 else if (b2.length == 0) 1 else 3 + } - final class ByteStrings private (private val bytestrings: Vector[ByteString1]) extends ByteString { + final class ByteStrings private (val bytestrings: Vector[ByteString1], val length: Int) extends ByteString { def apply(idx: Int): Byte = if (0 <= idx && idx < length) { @@ -148,37 +165,39 @@ object ByteString { bytestrings(pos)(idx - seen) } else throw new IndexOutOfBoundsException(idx.toString) - val length: Int = (0 /: bytestrings)(_ + _.length) - override def slice(from: Int, until: Int): ByteString = { val start = math.max(from, 0) val end = math.min(until, length) if (end <= start) ByteString.empty else { + val iter = bytestrings.iterator + var cur = iter.next var pos = 0 var seen = 0 - while (from >= seen + bytestrings(pos).length) { - seen += bytestrings(pos).length + while (from >= seen + cur.length) { + seen += cur.length pos += 1 + cur = iter.next } val startpos = pos val startidx = start - seen - while (until > seen + bytestrings(pos).length) { - seen += bytestrings(pos).length + while (until > seen + cur.length) { + seen += cur.length pos += 1 + cur = iter.next } val endpos = pos val endidx = end - seen if (startpos == endpos) - bytestrings(startpos).slice(startidx, endidx) + cur.slice(startidx, endidx) else { val first = bytestrings(startpos).drop(startidx).asInstanceOf[ByteString1] - val last = bytestrings(endpos).take(endidx).asInstanceOf[ByteString1] + val last = cur.take(endidx).asInstanceOf[ByteString1] if ((endpos - startpos) == 1) - new ByteStrings(Vector(first, last)) + new ByteStrings(Vector(first, last), until - from) else - new ByteStrings(first +: bytestrings.slice(startpos + 1, endpos) :+ last) + new ByteStrings(first +: bytestrings.slice(startpos + 1, endpos) :+ last, until - from) } } } @@ -200,7 +219,16 @@ object ByteString { def asByteBuffer: ByteBuffer = compact.asByteBuffer - def utf8String: String = compact.utf8String + def decodeString(charset: String): String = compact.decodeString(charset) + + def copyToBuffer(buffer: ByteBuffer): Int = { + val copyLength = math.min(buffer.remaining, length) + val iter = bytestrings.iterator + while (iter.hasNext && buffer.hasRemaining) { + iter.next.copyToBuffer(buffer) + } + copyLength + } } } @@ -208,9 +236,92 @@ object ByteString { sealed trait ByteString extends IndexedSeq[Byte] with IndexedSeqOptimized[Byte, ByteString] { override protected[this] def newBuilder = ByteString.newBuilder def ++(that: ByteString): ByteString + def copyToBuffer(buffer: ByteBuffer): Int def compact: ByteString def asByteBuffer: ByteBuffer - def toByteBuffer: ByteBuffer = ByteBuffer.wrap(toArray) - def utf8String: String - def mapI(f: Byte ⇒ Int): ByteString = map(f andThen (_.toByte)) + final def toByteBuffer: ByteBuffer = ByteBuffer.wrap(toArray) + final def utf8String: String = decodeString("UTF-8") + def decodeString(charset: String): String + final def mapI(f: Byte ⇒ Int): ByteString = map(f andThen (_.toByte)) +} + +final class ByteStringBuilder extends Builder[Byte, ByteString] { + import ByteString.{ ByteString1, ByteStrings } + private var _length = 0 + private val _builder = new VectorBuilder[ByteString1]() + private var _temp: Array[Byte] = _ + private var _tempLength = 0 + private var _tempCapacity = 0 + + private def clearTemp() { + if (_tempLength > 0) { + val arr = new Array[Byte](_tempLength) + Array.copy(_temp, 0, arr, 0, _tempLength) + _builder += ByteString1(arr) + _tempLength = 0 + } + } + + private def resizeTemp(size: Int) { + val newtemp = new Array[Byte](size) + if (_tempLength > 0) Array.copy(_temp, 0, newtemp, 0, _tempLength) + _temp = newtemp + } + + private def ensureTempSize(size: Int) { + if (_tempCapacity < size || _tempCapacity == 0) { + var newSize = if (_tempCapacity == 0) 16 else _tempCapacity * 2 + while (newSize < size) newSize *= 2 + resizeTemp(newSize) + } + } + + def +=(elem: Byte): this.type = { + ensureTempSize(_tempLength + 1) + _temp(_tempLength) = elem + _tempLength += 1 + _length += 1 + this + } + + override def ++=(xs: TraversableOnce[Byte]): this.type = { + xs match { + case b: ByteString1 ⇒ + clearTemp() + _builder += b + _length += b.length + case bs: ByteStrings ⇒ + clearTemp() + _builder ++= bs.bytestrings + _length += bs.length + case xs: WrappedArray.ofByte ⇒ + clearTemp() + _builder += ByteString1(xs.array.clone) + _length += xs.length + case _: collection.IndexedSeq[_] ⇒ + ensureTempSize(_tempLength + xs.size) + xs.copyToArray(_temp, _tempLength) + case _ ⇒ + super.++=(xs) + } + this + } + + def clear() { + _builder.clear + _length = 0 + _tempLength = 0 + } + + def result: ByteString = + if (_length == 0) ByteString.empty + else { + clearTemp() + val bytestrings = _builder.result + if (bytestrings.size == 1) + bytestrings.head + else + ByteStrings(bytestrings, _length) + } + } From 5bf8ca5b43cb5ede53e93b96093fe5309f8677ff Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Sat, 31 Dec 2011 15:31:08 -0700 Subject: [PATCH 003/152] Use correct implicits --- .../src/test/scala/akka/actor/IOActor.scala | 8 ++++ akka-actor/src/main/scala/akka/actor/IO.scala | 43 +++++++++---------- 2 files changed, 29 insertions(+), 22 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala index 06d3235409..5c2c162f25 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala @@ -14,6 +14,8 @@ object IOActorSpec { class SimpleEchoServer(host: String, port: Int, started: TestLatch) extends Actor { + implicit val system = context.system + IO listen (host, port) started.open @@ -44,6 +46,8 @@ object IOActorSpec { class SimpleEchoClient(host: String, port: Int) extends Actor { + implicit val system = context.system + val socket = IO connect (host, port) val state = IO.IterateeRef.sync() @@ -87,6 +91,8 @@ object IOActorSpec { // Basic Redis-style protocol class KVStore(host: String, port: Int, started: TestLatch) extends Actor { + implicit val system = context.system + val state = IO.IterateeRef.Map.sync[IO.Handle]() var kvs: Map[String, String] = Map.empty @@ -151,6 +157,8 @@ object IOActorSpec { class KVClient(host: String, port: Int) extends Actor { + implicit val system = context.system + val socket = IO connect (host, port) val state = IO.IterateeRef.sync() diff --git a/akka-actor/src/main/scala/akka/actor/IO.scala b/akka-actor/src/main/scala/akka/actor/IO.scala index 89922c94ca..bf4a8fc84b 100644 --- a/akka-actor/src/main/scala/akka/actor/IO.scala +++ b/akka-actor/src/main/scala/akka/actor/IO.scala @@ -3,6 +3,7 @@ */ package akka.actor +import akka.dispatch.{ Future, ExecutionContext } import akka.util.ByteString import java.net.InetSocketAddress import java.io.IOException @@ -74,37 +75,37 @@ object IO { case class Read(handle: ReadHandle, bytes: ByteString) extends IOMessage case class Write(handle: WriteHandle, bytes: ByteString) extends IOMessage - def listen(address: InetSocketAddress)(implicit context: ActorContext, owner: ActorRef): ServerHandle = { - val ioManager = IOManager.start()(context.system) + def listen(address: InetSocketAddress)(implicit system: ActorSystem, owner: ActorRef): ServerHandle = { + val ioManager = IOManager.start()(system) val server = ServerHandle(owner, ioManager) ioManager ! Listen(server, address) server } - def listen(host: String, port: Int)(implicit context: ActorContext, owner: ActorRef): ServerHandle = - listen(new InetSocketAddress(host, port))(context, owner) + def listen(host: String, port: Int)(implicit system: ActorSystem, owner: ActorRef): ServerHandle = + listen(new InetSocketAddress(host, port))(system, owner) - def listen(address: InetSocketAddress, owner: ActorRef)(implicit context: ActorContext): ServerHandle = - listen(address)(context, owner) + def listen(address: InetSocketAddress, owner: ActorRef)(implicit system: ActorSystem): ServerHandle = + listen(address)(system, owner) - def listen(host: String, port: Int, owner: ActorRef)(implicit context: ActorContext): ServerHandle = - listen(new InetSocketAddress(host, port))(context, owner) + def listen(host: String, port: Int, owner: ActorRef)(implicit system: ActorSystem): ServerHandle = + listen(new InetSocketAddress(host, port))(system, owner) - def connect(address: InetSocketAddress)(implicit context: ActorContext, owner: ActorRef): SocketHandle = { - val ioManager = IOManager.start()(context.system) + def connect(address: InetSocketAddress)(implicit system: ActorSystem, owner: ActorRef): SocketHandle = { + val ioManager = IOManager.start()(system) val socket = SocketHandle(owner, ioManager) ioManager ! Connect(socket, address) socket } - def connect(host: String, port: Int)(implicit context: ActorContext, owner: ActorRef): SocketHandle = - connect(new InetSocketAddress(host, port))(context, owner) + def connect(host: String, port: Int)(implicit system: ActorSystem, owner: ActorRef): SocketHandle = + connect(new InetSocketAddress(host, port))(system, owner) - def connect(address: InetSocketAddress, owner: ActorRef)(implicit context: ActorContext): SocketHandle = - connect(address)(context, owner) + def connect(address: InetSocketAddress, owner: ActorRef)(implicit system: ActorSystem): SocketHandle = + connect(address)(system, owner) - def connect(host: String, port: Int, owner: ActorRef)(implicit context: ActorContext): SocketHandle = - connect(new InetSocketAddress(host, port))(context, owner) + def connect(host: String, port: Int, owner: ActorRef)(implicit system: ActorSystem): SocketHandle = + connect(new InetSocketAddress(host, port))(system, owner) sealed trait Input { def ++(that: Input): Input @@ -190,8 +191,8 @@ object IO { def sync[A](initial: Iteratee[A]): IterateeRefSync[A] = new IterateeRefSync(initial) def sync(): IterateeRefSync[Unit] = new IterateeRefSync(Iteratee.unit) - def async[A](initial: Iteratee[A])(implicit app: ActorSystem): IterateeRefAsync[A] = new IterateeRefAsync(initial) - def async()(implicit app: ActorSystem): IterateeRefAsync[Unit] = new IterateeRefAsync(Iteratee.unit) + def async[A](initial: Iteratee[A])(implicit executor: ExecutionContext): IterateeRefAsync[A] = new IterateeRefAsync(initial) + def async()(implicit executor: ExecutionContext): IterateeRefAsync[Unit] = new IterateeRefAsync(Iteratee.unit) 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)) @@ -203,7 +204,7 @@ object IO { object Map { def apply[K, V](refFactory: ⇒ IterateeRef[V]): IterateeRef.Map[K, V] = new Map(refFactory) def sync[K](): IterateeRef.Map[K, Unit] = new Map(IterateeRef.sync()) - def async[K]()(implicit app: ActorSystem): IterateeRef.Map[K, Unit] = new Map(IterateeRef.async()) + def async[K]()(implicit executor: ExecutionContext): IterateeRef.Map[K, Unit] = new Map(IterateeRef.async()) } } @@ -232,8 +233,7 @@ object IO { def value: (Iteratee[A], Input) = _value } - final class IterateeRefAsync[A](initial: Iteratee[A])(implicit app: ActorSystem) extends IterateeRef[A] { - import akka.dispatch.Future + 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 { case (iter, chunk @ Chunk(bytes)) if bytes.nonEmpty ⇒ (iter flatMap f)(chunk) @@ -421,7 +421,6 @@ object IOManager { def stop()(implicit system: ActorSystem): Unit = { // TODO: send shutdown message to IOManager } - } final class WriteBuffer(bufferSize: Int) { From a44da38e2b202f57bbaee1cb96697be0b90ddb15 Mon Sep 17 00:00:00 2001 From: Nikolay Botev Date: Sat, 17 Dec 2011 17:18:50 -0800 Subject: [PATCH 004/152] ask-2.0 --- .../src/main/scala/akka/actor/ActorRef.scala | 85 ++----------------- .../scala/akka/actor/ActorRefProvider.scala | 2 + .../src/main/scala/akka/actor/package.scala | 3 + .../src/main/scala/akka/dispatch/Future.scala | 79 +++++++++++++++++ .../docs/actor/FaultHandlingTestBase.java | 19 +++-- .../docs/actor/UntypedActorDocTestBase.java | 3 +- .../akka/docs/future/FutureDocTestBase.java | 3 +- .../code/akka/docs/actor/ActorDocSpec.scala | 5 +- .../code/akka/docs/future/FutureDocSpec.scala | 21 ++--- .../akka/docs/routing/RouterTypeExample.scala | 4 +- .../akka/docs/testkit/TestkitDocSpec.scala | 3 +- .../akka/remote/RemoteActorRefProvider.scala | 13 +-- .../scala/akka/testkit/TestActorRef.scala | 2 +- 13 files changed, 122 insertions(+), 120 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index f1378db41a..c4009fdbb4 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -104,33 +104,6 @@ abstract class ActorRef extends java.lang.Comparable[ActorRef] with Serializable */ final def tell(msg: Any, sender: ActorRef): Unit = this.!(msg)(sender) - /** - * Akka Java API. - * - * Sends a message asynchronously returns a future holding the eventual reply message. - * The Future will be completed with an [[akka.actor.AskTimeoutException]] after the given - * timeout has expired. - * - * NOTE: - * Use this method with care. In most cases it is better to use 'tell' together with the sender - * parameter to implement non-blocking request/response message exchanges. - * - * If you are sending messages using ask and using blocking operations on the Future, such as - * 'get', then you have to use getContext().sender().tell(...) - * in the target actor to send a reply message to the original sender, and thereby completing the Future, - * otherwise the sender will block until the timeout expires. - * - * When using future callbacks, inside actors you need to carefully avoid closing over - * the containing actor’s reference, i.e. do not call methods or access mutable state - * on the enclosing actor from within the callback. This would break the actor - * encapsulation and may introduce synchronization bugs and race conditions because - * the callback will be scheduled concurrently to the enclosing actor. Unfortunately - * there is not yet a way to detect these illegal accesses at compile time. - */ - def ask(message: AnyRef, timeout: Timeout): Future[AnyRef] = ?(message, timeout).asInstanceOf[Future[AnyRef]] - - def ask(message: AnyRef, timeoutMillis: Long): Future[AnyRef] = ask(message, new Timeout(timeoutMillis)) - /** * Forwards the message and passes the original sender actor as the sender. *

@@ -179,35 +152,6 @@ trait ScalaActorRef { ref: ActorRef ⇒ */ def !(message: Any)(implicit sender: ActorRef = null): Unit - /** - * Sends a message asynchronously, returning a future which may eventually hold the reply. - * The Future will be completed with an [[akka.actor.AskTimeoutException]] after the given - * timeout has expired. - * - * NOTE: - * Use this method with care. In most cases it is better to use '!' together with implicit or explicit - * sender parameter to implement non-blocking request/response message exchanges. - * - * If you are sending messages using ask and using blocking operations on the Future, such as - * 'get', then you have to use getContext().sender().tell(...) - * in the target actor to send a reply message to the original sender, and thereby completing the Future, - * otherwise the sender will block until the timeout expires. - * - * When using future callbacks, inside actors you need to carefully avoid closing over - * the containing actor’s reference, i.e. do not call methods or access mutable state - * on the enclosing actor from within the callback. This would break the actor - * encapsulation and may introduce synchronization bugs and race conditions because - * the callback will be scheduled concurrently to the enclosing actor. Unfortunately - * there is not yet a way to detect these illegal accesses at compile time. - */ - def ?(message: Any)(implicit timeout: Timeout): Future[Any] - - /** - * Sends a message asynchronously, returning a future which may eventually hold the reply. - * The implicit parameter with the default value is just there to disambiguate it from the version that takes the - * implicit timeout - */ - def ?(message: Any, timeout: Timeout)(implicit ignore: Int = 0): Future[Any] = ?(message)(timeout) } /** @@ -236,6 +180,7 @@ private[akka] abstract class InternalActorRef extends ActorRef with ScalaActorRe def stop(): Unit def sendSystemMessage(message: SystemMessage): Unit def getParent: InternalActorRef + def provider: ActorRefProvider /** * Obtain ActorRef by possibly traversing the actor tree or looking it up at * some provider-specific location. This method shall return the end result, @@ -321,6 +266,8 @@ private[akka] class LocalActorRef private[akka] ( def getParent: InternalActorRef = actorCell.parent + def provider = actorCell.provider + /** * Method for looking up a single child beneath this actor. Override in order * to inject “synthetic” actor paths like “/temp”. @@ -365,17 +312,6 @@ private[akka] class LocalActorRef private[akka] ( def !(message: Any)(implicit sender: ActorRef = null): Unit = actorCell.tell(message, sender) - def ?(message: Any)(implicit timeout: Timeout): Future[Any] = { - actorCell.provider.ask(timeout) match { - case Some(a) ⇒ - this.!(message)(a) - a.result - case None ⇒ - this.!(message)(null) - Promise[Any]()(actorCell.system.dispatcher) - } - } - def restart(cause: Throwable): Unit = actorCell.restart(cause) @throws(classOf[java.io.ObjectStreamException]) @@ -405,6 +341,8 @@ case class SerializedActorRef(path: String) { trait MinimalActorRef extends InternalActorRef with LocalRef { def getParent: InternalActorRef = Nobody + def provider: ActorRefProvider = + throw new UnsupportedOperationException("Not supported for [%s]".format(getClass.getName)) def getChild(names: Iterator[String]): InternalActorRef = { val dropped = names.dropWhile(_.isEmpty) if (dropped.isEmpty) this @@ -420,9 +358,6 @@ trait MinimalActorRef extends InternalActorRef with LocalRef { def !(message: Any)(implicit sender: ActorRef = null): Unit = () - def ?(message: Any)(implicit timeout: Timeout): Future[Any] = - throw new UnsupportedOperationException("Not supported for [%s]".format(getClass.getName)) - def sendSystemMessage(message: SystemMessage): Unit = () def restart(cause: Throwable): Unit = () @@ -471,13 +406,6 @@ class DeadLetterActorRef(val eventStream: EventStream) extends MinimalActorRef { case _ ⇒ eventStream.publish(DeadLetter(message, sender, this)) } - override def ?(message: Any)(implicit timeout: Timeout): Future[Any] = { - eventStream.publish(DeadLetter(message, this, this)) - // leave this in: guard with good visibility against really stupid/weird errors - assert(brokenPromise != null) - brokenPromise - } - @throws(classOf[java.io.ObjectStreamException]) override protected def writeReplace(): AnyRef = DeadLetterActorRef.serialized } @@ -558,9 +486,6 @@ class AskActorRef( case _ ⇒ } - override def ?(message: Any)(implicit timeout: Timeout): Future[Any] = - Promise.failed(new UnsupportedOperationException("Ask/? is not supported for [%s]".format(getClass.getName)))(dispatcher) - override def isTerminated = result.isCompleted override def stop(): Unit = if (running.getAndSet(false)) { diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index d940aa2c20..861a234db3 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -56,6 +56,8 @@ trait ActorRefProvider { def settings: ActorSystem.Settings + def dispatcher: MessageDispatcher + /** * Initialization of an ActorRefProvider happens in two steps: first * construction of the object with settings, eventStream, scheduler, etc. diff --git a/akka-actor/src/main/scala/akka/actor/package.scala b/akka-actor/src/main/scala/akka/actor/package.scala index cfe5bc1b0d..0b06470c79 100644 --- a/akka-actor/src/main/scala/akka/actor/package.scala +++ b/akka-actor/src/main/scala/akka/actor/package.scala @@ -8,6 +8,9 @@ package object actor { implicit def actorRef2Scala(ref: ActorRef): ScalaActorRef = ref.asInstanceOf[ScalaActorRef] implicit def scala2ActorRef(ref: ScalaActorRef): ActorRef = ref.asInstanceOf[ActorRef] + implicit def actorRef2Askable(actorRef: ActorRef) = new dispatch.AskableActorRef(actorRef) + implicit def askable2ActorRef(askable: dispatch.AskableActorRef) = askable.actorRef + type Uuid = com.eaio.uuid.UUID def newUuid(): Uuid = new Uuid() diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index fea97fbaf3..c70ec32c43 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -24,6 +24,7 @@ import java.util.concurrent.atomic.{ AtomicReferenceFieldUpdater, AtomicInteger, import akka.dispatch.Await.CanAwait import java.util.concurrent._ import akka.actor.ActorSystem +import akka.actor.{ ActorRef, InternalActorRef } object Await { sealed trait CanAwait @@ -53,6 +54,24 @@ object Await { */ object Futures { + def ask(actor: ActorRef, message: Any)(implicit timeout: Timeout): Future[Any] = { + val provider = actor.asInstanceOf[InternalActorRef].provider + provider.ask(timeout) match { + case Some(a) ⇒ + actor.!(message)(a) + a.result + case None ⇒ + actor.!(message)(null) + Promise[Any]()(provider.dispatcher) + } + } + + def ask(actor: ActorRef, message: Any, timeout: Timeout)(implicit ignore: Int = 0): Future[Any] = + ask(actor, message)(timeout) + + def ask(actor: ActorRef, message: Any, timeoutMillis: Long): Future[Any] = + ask(actor, message)(new Timeout(timeoutMillis)) + /** * Java API, equivalent to Future.apply */ @@ -134,6 +153,66 @@ object Futures { } } +final class AskableActorRef(val actorRef: ActorRef) { + + /** + * Akka Java API. + * + * Sends a message asynchronously returns a future holding the eventual reply message. + * The Future will be completed with an [[akka.actor.AskTimeoutException]] after the given + * timeout has expired. + * + * NOTE: + * Use this method with care. In most cases it is better to use 'tell' together with the sender + * parameter to implement non-blocking request/response message exchanges. + * + * If you are sending messages using ask and using blocking operations on the Future, such as + * 'get', then you have to use getContext().sender().tell(...) + * in the target actor to send a reply message to the original sender, and thereby completing the Future, + * otherwise the sender will block until the timeout expires. + * + * When using future callbacks, inside actors you need to carefully avoid closing over + * the containing actor’s reference, i.e. do not call methods or access mutable state + * on the enclosing actor from within the callback. This would break the actor + * encapsulation and may introduce synchronization bugs and race conditions because + * the callback will be scheduled concurrently to the enclosing actor. Unfortunately + * there is not yet a way to detect these illegal accesses at compile time. + */ + def ask(message: AnyRef, timeout: Timeout): Future[AnyRef] = ?(message, timeout).asInstanceOf[Future[AnyRef]] + + def ask(message: AnyRef, timeoutMillis: Long): Future[AnyRef] = ask(message, new Timeout(timeoutMillis)) + + /** + * Sends a message asynchronously, returning a future which may eventually hold the reply. + * The Future will be completed with an [[akka.actor.AskTimeoutException]] after the given + * timeout has expired. + * + * NOTE: + * Use this method with care. In most cases it is better to use '!' together with implicit or explicit + * sender parameter to implement non-blocking request/response message exchanges. + * + * If you are sending messages using ask and using blocking operations on the Future, such as + * 'get', then you have to use getContext().sender().tell(...) + * in the target actor to send a reply message to the original sender, and thereby completing the Future, + * otherwise the sender will block until the timeout expires. + * + * When using future callbacks, inside actors you need to carefully avoid closing over + * the containing actor’s reference, i.e. do not call methods or access mutable state + * on the enclosing actor from within the callback. This would break the actor + * encapsulation and may introduce synchronization bugs and race conditions because + * the callback will be scheduled concurrently to the enclosing actor. Unfortunately + * there is not yet a way to detect these illegal accesses at compile time. + */ + def ?(message: Any)(implicit timeout: Timeout): Future[Any] = Futures.ask(actorRef, message) + + /** + * Sends a message asynchronously, returning a future which may eventually hold the reply. + * The implicit parameter with the default value is just there to disambiguate it from the version that takes the + * implicit timeout + */ + def ?(message: Any, timeout: Timeout)(implicit ignore: Int = 0): Future[Any] = ?(message)(timeout) +} + object Future { /** diff --git a/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java b/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java index 132dc990ee..1bc3b40c4b 100644 --- a/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java +++ b/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java @@ -13,6 +13,7 @@ import akka.actor.Props; import akka.actor.Terminated; import akka.actor.UntypedActor; import akka.dispatch.Await; +import static akka.dispatch.Futures.ask; import akka.util.Duration; import akka.testkit.AkkaSpec; import akka.testkit.TestProbe; @@ -126,19 +127,19 @@ public class FaultHandlingTestBase { //#create Props superprops = new Props(Supervisor.class).withFaultHandler(strategy); ActorRef supervisor = system.actorOf(superprops, "supervisor"); - ActorRef child = (ActorRef) Await.result(supervisor.ask(new Props(Child.class), 5000), timeout); + ActorRef child = (ActorRef) Await.result(ask(supervisor, new Props(Child.class), 5000), timeout); //#create //#resume child.tell(42); - assert Await.result(child.ask("get", 5000), timeout).equals(42); + assert Await.result(ask(child, "get", 5000), timeout).equals(42); child.tell(new ArithmeticException()); - assert Await.result(child.ask("get", 5000), timeout).equals(42); + assert Await.result(ask(child, "get", 5000), timeout).equals(42); //#resume //#restart child.tell(new NullPointerException()); - assert Await.result(child.ask("get", 5000), timeout).equals(0); + assert Await.result(ask(child, "get", 5000), timeout).equals(0); //#restart //#stop @@ -149,9 +150,9 @@ public class FaultHandlingTestBase { //#stop //#escalate-kill - child = (ActorRef) Await.result(supervisor.ask(new Props(Child.class), 5000), timeout); + child = (ActorRef) Await.result(ask(supervisor, new Props(Child.class), 5000), timeout); probe.watch(child); - assert Await.result(child.ask("get", 5000), timeout).equals(0); + assert Await.result(ask(child, "get", 5000), timeout).equals(0); child.tell(new Exception()); probe.expectMsg(new Terminated(child)); //#escalate-kill @@ -159,11 +160,11 @@ public class FaultHandlingTestBase { //#escalate-restart superprops = new Props(Supervisor2.class).withFaultHandler(strategy); supervisor = system.actorOf(superprops, "supervisor2"); - child = (ActorRef) Await.result(supervisor.ask(new Props(Child.class), 5000), timeout); + child = (ActorRef) Await.result(ask(supervisor, new Props(Child.class), 5000), timeout); child.tell(23); - assert Await.result(child.ask("get", 5000), timeout).equals(23); + assert Await.result(ask(child, "get", 5000), timeout).equals(23); child.tell(new Exception()); - assert Await.result(child.ask("get", 5000), timeout).equals(0); + assert Await.result(ask(child, "get", 5000), timeout).equals(0); //#escalate-restart //#testkit } diff --git a/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java b/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java index d442ae6461..be164aa850 100644 --- a/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java +++ b/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java @@ -11,6 +11,7 @@ import akka.actor.Props; //#import-future import akka.dispatch.Future; +import akka.dispatch.Futures; import akka.dispatch.Await; import akka.util.Duration; import akka.util.Timeout; @@ -117,7 +118,7 @@ public class UntypedActorDocTestBase { }), "myactor"); //#using-ask - Future future = myActor.ask("Hello", 1000); + Future future = Futures.ask(myActor, "Hello", 1000); Object result = Await.result(future, Duration.create(1, TimeUnit.SECONDS)); //#using-ask system.shutdown(); diff --git a/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java b/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java index c3278f23bd..956e72eab9 100644 --- a/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java +++ b/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java @@ -53,7 +53,6 @@ import akka.actor.Status.Failure; import akka.actor.ActorSystem; import akka.actor.UntypedActor; import akka.actor.ActorRef; -import akka.docs.actor.MyUntypedActor; import akka.actor.Props; import akka.dispatch.Futures; @@ -79,7 +78,7 @@ public class FutureDocTestBase { String msg = "hello"; //#ask-blocking Timeout timeout = system.settings().ActorTimeout(); - Future future = actor.ask(msg, timeout); + Future future = Futures.ask(actor, msg, timeout); String result = (String) Await.result(future, timeout.duration()); //#ask-blocking assertEquals("HELLO", result); diff --git a/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala b/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala index cdba3d07f3..3907e6cf47 100644 --- a/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala @@ -8,6 +8,7 @@ import akka.actor.Actor import akka.actor.Props import akka.event.Logging import akka.dispatch.Future +import akka.dispatch.Futures //#imports1 @@ -229,10 +230,10 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) { val myActor = system.actorOf(Props(new MyActor), name = "myactor") implicit val timeout = system.settings.ActorTimeout - val future = myActor ? "hello" + val future = Futures.ask(myActor, "hello") for (x ← future) println(x) //Prints "hello" - val result: Future[Int] = for (x ← (myActor ? 3).mapTo[Int]) yield { 2 * x } + val result: Future[Int] = for (x ← Futures.ask(myActor, 3).mapTo[Int]) yield { 2 * x } //#using-ask system.stop(myActor) diff --git a/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala b/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala index 1b6a755ede..bb96a4e3a0 100644 --- a/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala @@ -10,6 +10,7 @@ import akka.actor.Actor import akka.actor.Props import akka.actor.Status.Failure import akka.dispatch.Future +import akka.dispatch.Futures import akka.dispatch.Await import akka.util.duration._ import akka.dispatch.Promise @@ -46,7 +47,7 @@ class FutureDocSpec extends AkkaSpec { import akka.dispatch.Await implicit val timeout = system.settings.ActorTimeout - val future = actor ? msg + val future = Futures.ask(actor, msg) val result = Await.result(future, timeout.duration).asInstanceOf[String] //#ask-blocking result must be("HELLO") @@ -59,7 +60,7 @@ class FutureDocSpec extends AkkaSpec { //#map-to import akka.dispatch.Future - val future: Future[String] = (actor ? msg).mapTo[String] + val future: Future[String] = Futures.ask(actor, msg).mapTo[String] //#map-to Await.result(future, timeout.duration) must be("HELLO") } @@ -149,13 +150,13 @@ class FutureDocSpec extends AkkaSpec { import akka.dispatch.Await //#composing-wrong - val f1 = actor1 ? msg1 - val f2 = actor2 ? msg2 + val f1 = Futures.ask(actor1, msg1) + val f2 = Futures.ask(actor2, msg2) val a = Await.result(f1, 1 second).asInstanceOf[Int] val b = Await.result(f2, 1 second).asInstanceOf[Int] - val f3 = actor3 ? (a + b) + val f3 = Futures.ask(actor3, (a + b)) val result = Await.result(f3, 1 second).asInstanceOf[Int] //#composing-wrong @@ -172,13 +173,13 @@ class FutureDocSpec extends AkkaSpec { import akka.dispatch.Await //#composing - val f1 = actor1 ? msg1 - val f2 = actor2 ? msg2 + val f1 = Futures.ask(actor1, msg1) + val f2 = Futures.ask(actor2, msg2) val f3 = for { a ← f1.mapTo[Int] b ← f2.mapTo[Int] - c ← (actor3 ? (a + b)).mapTo[Int] + c ← Futures.ask(actor3, (a + b)).mapTo[Int] } yield c val result = Await.result(f3, 1 second).asInstanceOf[Int] @@ -191,7 +192,7 @@ class FutureDocSpec extends AkkaSpec { val oddActor = system.actorOf(Props[OddActor]) //#sequence-ask // oddActor returns odd numbers sequentially from 1 as a List[Future[Int]] - val listOfFutures = List.fill(100)((oddActor ? GetNext).mapTo[Int]) + val listOfFutures = List.fill(100)(Futures.ask(oddActor, GetNext).mapTo[Int]) // now we have a Future[List[Int]] val futureList = Future.sequence(listOfFutures) @@ -239,7 +240,7 @@ class FutureDocSpec extends AkkaSpec { val actor = system.actorOf(Props[MyActor]) val msg1 = -1 //#recover - val future = actor ? msg1 recover { + val future = Futures.ask(actor, msg1) recover { case e: ArithmeticException ⇒ 0 } //#recover diff --git a/akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala b/akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala index 63338e8357..c366c011c1 100644 --- a/akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala +++ b/akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala @@ -7,7 +7,7 @@ import akka.routing.{ ScatterGatherFirstCompletedRouter, BroadcastRouter, Random import annotation.tailrec import akka.actor.{ Props, Actor } import akka.util.duration._ -import akka.dispatch.Await +import akka.dispatch.{ Futures, Await } case class FibonacciNumber(nbr: Int) @@ -71,7 +71,7 @@ class ParentActor extends Actor { Props[FibonacciActor].withRouter(ScatterGatherFirstCompletedRouter(within = 2 seconds)), "router") implicit val timeout = context.system.settings.ActorTimeout - val futureResult = scatterGatherFirstCompletedRouter ? FibonacciNumber(10) + val futureResult = Futures.ask(scatterGatherFirstCompletedRouter, FibonacciNumber(10)) val result = Await.result(futureResult, timeout.duration) //#scatterGatherFirstCompletedRouter println("The result of calculating Fibonacci for 10 is %d".format(result)) diff --git a/akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala b/akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala index 0678860ba9..c646f9ea54 100644 --- a/akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala @@ -7,6 +7,7 @@ package akka.docs.testkit import akka.testkit.TestProbe import akka.util.duration._ import akka.actor._ +import akka.dispatch.Futures //#imports-test-probe @@ -204,7 +205,7 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender { import akka.util.duration._ //#test-probe-reply val probe = TestProbe() - val future = probe.ref ? "hello" + val future = Futures.ask(probe.ref, "hello") probe.expectMsg(0 millis, "hello") // TestActor runs on CallingThreadDispatcher probe.sender ! "world" assert(future.isCompleted && future.value == Some(Right("world"))) diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 5b748098ac..6f11d4321c 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -160,7 +160,7 @@ trait RemoteRef extends ActorRefScope { * This reference is network-aware (remembers its origin) and immutable. */ private[akka] class RemoteActorRef private[akka] ( - provider: RemoteActorRefProvider, + override val provider: RemoteActorRefProvider, remote: RemoteSupport[ParsedTransportAddress], val path: ActorPath, val getParent: InternalActorRef, @@ -185,17 +185,6 @@ private[akka] class RemoteActorRef private[akka] ( override def !(message: Any)(implicit sender: ActorRef = null): Unit = remote.send(message, Option(sender), this, loader) - override def ?(message: Any)(implicit timeout: Timeout): Future[Any] = { - provider.ask(timeout) match { - case Some(a) ⇒ - this.!(message)(a) - a.result - case None ⇒ - this.!(message)(null) - Promise[Any]()(provider.dispatcher) - } - } - def suspend(): Unit = sendSystemMessage(Suspend()) def resume(): Unit = sendSystemMessage(Resume()) diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala index eaeecf7487..2d52f21226 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala @@ -71,7 +71,7 @@ class TestActorRef[T <: Actor]( underlying.actor.asInstanceOf[T] match { case null ⇒ val t = underlying.system.settings.ActorTimeout - Await.result(?(InternalGetActor)(t), t.duration).asInstanceOf[T] + Await.result(this.?(InternalGetActor)(t), t.duration).asInstanceOf[T] case ref ⇒ ref } } From 4270b6f71ba439e2d46d063c1c9f4029459d76c8 Mon Sep 17 00:00:00 2001 From: Nikolay Botev Date: Sat, 17 Dec 2011 22:08:14 -0800 Subject: [PATCH 005/152] ask 2.1 --- .../src/main/scala/akka/actor/ActorRef.scala | 5 ++-- .../scala/akka/actor/ActorRefProvider.scala | 25 ++++++++++++++++--- .../src/main/scala/akka/dispatch/Future.scala | 6 ++--- .../src/main/scala/akka/routing/Routing.scala | 7 ++++-- .../akka/remote/RemoteActorRefProvider.scala | 2 +- .../akka/remote/RemoteCommunicationSpec.scala | 4 +-- 6 files changed, 34 insertions(+), 15 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index c4009fdbb4..b689aa5370 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -466,14 +466,13 @@ class AskTimeoutException(message: String, cause: Throwable) extends TimeoutExce def this(message: String) = this(message, null: Throwable) } -class AskActorRef( +private[akka] final class PromiseActorRef( val path: ActorPath, override val getParent: InternalActorRef, - val dispatcher: MessageDispatcher, + private final val result: Promise[Any], val deathWatch: DeathWatch) extends MinimalActorRef { final val running = new AtomicBoolean(true) - final val result = Promise[Any]()(dispatcher) override def !(message: Any)(implicit sender: ActorRef = null): Unit = if (running.get) message match { case Status.Success(r) ⇒ result.success(r) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 861a234db3..0b3215cd60 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -104,7 +104,7 @@ trait ActorRefProvider { * Create AskActorRef and register it properly so it can be serialized/deserialized; * caller needs to send the message. */ - def ask(within: Timeout): Option[AskActorRef] + def ask(result: Promise[Any], within: Timeout): Option[ActorRef] /** * This Future is completed upon termination of this ActorRefProvider, which @@ -494,15 +494,14 @@ class LocalActorRefProvider( } } - def ask(within: Timeout): Option[AskActorRef] = { + def ask(result: Promise[Any], within: Timeout): Option[ActorRef] = { (if (within == null) settings.ActorTimeout else within) match { case t if t.duration.length <= 0 ⇒ None case t ⇒ val path = tempPath() val name = path.name - val a = new AskActorRef(path, tempContainer, dispatcher, deathWatch) + val a = new PromiseActorRef(path, tempContainer, result, deathWatch) tempContainer.addChild(name, a) - val result = a.result val f = dispatcher.prerequisites.scheduler.scheduleOnce(t.duration) { result.failure(new AskTimeoutException("Timed out")) } result onComplete { _ ⇒ try { a.stop(); f.cancel() } @@ -510,6 +509,24 @@ class LocalActorRefProvider( } Some(a) + + // Alternative implementation: + // Create a full-blown actor to complete the promise. + // This would also work but not as efficient as PromiseActorRef. + //val b = actorOf(system, Props(new Actor { + // def receive = { + // case Status.Success(r) ⇒ result.success(r) + // case Status.Failure(f) ⇒ result.failure(f) + // case other ⇒ result.success(other) + // } + //}), systemGuardian, systemGuardian.path / "promise" / tempName(), false, None) + //val ff = system.scheduler.scheduleOnce(t.duration) { b.stop() } + //result onComplete { _ ⇒ + // b.stop() + // ff.cancel() + //} + // + //Some(b) } } } diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index c70ec32c43..fc4225600d 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -56,14 +56,14 @@ object Futures { def ask(actor: ActorRef, message: Any)(implicit timeout: Timeout): Future[Any] = { val provider = actor.asInstanceOf[InternalActorRef].provider - provider.ask(timeout) match { + val promise = Promise[Any]()(provider.dispatcher) + provider.ask(promise, timeout) match { case Some(a) ⇒ actor.!(message)(a) - a.result case None ⇒ actor.!(message)(null) - Promise[Any]()(provider.dispatcher) } + promise } def ask(actor: ActorRef, message: Any, timeout: Timeout)(implicit ignore: Int = 0): Future[Any] = diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index 0c02952b3e..36f5fe9670 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -8,6 +8,7 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.JavaConversions._ import akka.util.{ Duration, Timeout } import akka.config.ConfigurationException +import akka.dispatch.Promise /** * A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to @@ -405,8 +406,10 @@ trait ScatterGatherFirstCompletedLike { this: RouterConfig ⇒ { case (sender, message) ⇒ - val asker = context.asInstanceOf[ActorCell].systemImpl.provider.ask(Timeout(within)).get - asker.result.pipeTo(sender) + val provider: ActorRefProvider = context.asInstanceOf[ActorCell].systemImpl.provider + val promise = Promise[Any]()(provider.dispatcher) + val asker = provider.ask(promise, Timeout(within)).get + promise.pipeTo(sender) message match { case _ ⇒ toAll(asker, ref.routees) } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 6f11d4321c..5b38996d3b 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -138,7 +138,7 @@ class RemoteActorRefProvider( def actorFor(ref: InternalActorRef, path: Iterable[String]): InternalActorRef = local.actorFor(ref, path) - def ask(within: Timeout): Option[AskActorRef] = local.ask(within) + def ask(result: Promise[Any], within: Timeout): Option[ActorRef] = local.ask(result, within) /** * Using (checking out) actor on a specific node. diff --git a/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala index dd62ae48e2..0be7c0f361 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala @@ -82,8 +82,8 @@ akka { "support ask" in { Await.result(here ? "ping", timeout.duration) match { - case ("pong", s: AskActorRef) ⇒ // good - case m ⇒ fail(m + " was not (pong, AskActorRef)") + case ("pong", s: PromiseActorRef) ⇒ // good + case m ⇒ fail(m + " was not (pong, AskActorRef)") } } From 0558e11affc0cb8b70637f54d0d2f63b1dcdb135 Mon Sep 17 00:00:00 2001 From: Nikolay Botev Date: Sun, 18 Dec 2011 02:45:21 -0800 Subject: [PATCH 006/152] ask 2.2 --- akka-actor/src/main/scala/akka/actor/ActorSystem.scala | 10 +++++----- akka-actor/src/main/scala/akka/actor/TypedActor.scala | 6 +++--- akka-actor/src/main/scala/akka/event/Logging.scala | 4 ++-- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 0e95325ec3..3e9a0a91f4 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -289,7 +289,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor private[akka] def systemActorOf(props: Props, name: String): ActorRef = { implicit val timeout = settings.CreationTimeout - Await.result(systemGuardian ? CreateChild(props, name), timeout.duration) match { + Await.result(Futures.ask(systemGuardian, CreateChild(props, name)), timeout.duration) match { case ref: ActorRef ⇒ ref case ex: Exception ⇒ throw ex } @@ -297,7 +297,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor def actorOf(props: Props, name: String): ActorRef = { implicit val timeout = settings.CreationTimeout - Await.result(guardian ? CreateChild(props, name), timeout.duration) match { + Await.result(Futures.ask(guardian, CreateChild(props, name)), timeout.duration) match { case ref: ActorRef ⇒ ref case ex: Exception ⇒ throw ex } @@ -305,7 +305,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor def actorOf(props: Props): ActorRef = { implicit val timeout = settings.CreationTimeout - Await.result(guardian ? CreateRandomNameChild(props), timeout.duration) match { + Await.result(Futures.ask(guardian, CreateRandomNameChild(props)), timeout.duration) match { case ref: ActorRef ⇒ ref case ex: Exception ⇒ throw ex } @@ -317,8 +317,8 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor val guard = guardian.path val sys = systemGuardian.path path.parent match { - case `guard` ⇒ Await.result(guardian ? StopChild(actor), timeout.duration) - case `sys` ⇒ Await.result(systemGuardian ? StopChild(actor), timeout.duration) + case `guard` ⇒ Await.result(Futures.ask(guardian, StopChild(actor)), timeout.duration) + case `sys` ⇒ Await.result(Futures.ask(systemGuardian, StopChild(actor)), timeout.duration) case _ ⇒ actor.asInstanceOf[InternalActorRef].stop() } } diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index 3dc5d4c000..24ccda0f8d 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -398,15 +398,15 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi case _ ⇒ MethodCall(method, args) match { case m if m.isOneWay ⇒ actor ! m; null //Null return value - case m if m.returnsFuture_? ⇒ actor.?(m, timeout) + case m if m.returnsFuture_? ⇒ Futures.ask(actor, m)(timeout) case m if m.returnsJOption_? || m.returnsOption_? ⇒ - val f = actor.?(m, timeout) + val f = Futures.ask(actor, m)(timeout) (try { Await.ready(f, timeout.duration).value } catch { case _: TimeoutException ⇒ None }) match { case None | Some(Right(null)) ⇒ if (m.returnsJOption_?) JOption.none[Any] else None case Some(Right(joption: AnyRef)) ⇒ joption case Some(Left(ex)) ⇒ throw ex } - case m ⇒ Await.result(actor.?(m, timeout), timeout.duration).asInstanceOf[AnyRef] + case m ⇒ Await.result(Futures.ask(actor, m)(timeout), timeout.duration).asInstanceOf[AnyRef] } } } diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index bfd0f2a184..4099dc06b5 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -14,7 +14,7 @@ import akka.util.Timeout import java.util.concurrent.atomic.AtomicInteger import scala.util.control.NoStackTrace import java.util.concurrent.TimeoutException -import akka.dispatch.Await +import akka.dispatch.{ Await, Futures } object LoggingBus { implicit def fromActorSystem(system: ActorSystem): LoggingBus = system.eventStream @@ -158,7 +158,7 @@ trait LoggingBus extends ActorEventBus { val name = "log" + Extension(system).id() + "-" + simpleName(clazz) val actor = system.systemActorOf(Props(clazz), name) implicit val timeout = Timeout(3 seconds) - val response = try Await.result(actor ? InitializeLogger(this), timeout.duration) catch { + val response = try Await.result(Futures.ask(actor, InitializeLogger(this)), timeout.duration) catch { case _: TimeoutException ⇒ publish(Warning(simpleName(this), "Logger " + name + " did not respond within " + timeout + " to InitializeLogger(bus)")) } From 877075cdac274fbf2a7efba437dc3ecfddf5c03b Mon Sep 17 00:00:00 2001 From: Nikolay Botev Date: Mon, 19 Dec 2011 15:54:26 -0800 Subject: [PATCH 007/152] ask-2.3 --- .../main/scala/akka/actor/ActorSystem.scala | 14 +++++++----- .../src/main/scala/akka/actor/package.scala | 22 +++++++++++++++++++ .../src/main/scala/akka/dispatch/Future.scala | 12 ++++------ .../docs/transactor/TransactorDocTest.java | 9 ++++---- .../UntypedCoordinatedIncrementTest.java | 5 +++-- .../transactor/UntypedTransactorTest.java | 5 +++-- 6 files changed, 46 insertions(+), 21 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 3e9a0a91f4..e375f29e4b 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -287,9 +287,13 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor protected def systemImpl = this + @inline private def askAndAwait(actorRef: ActorRef, message: Any)(implicit timeout: akka.util.Timeout): Any = { + Await.result(Futures.ask(actorRef, message), timeout.duration) + } + private[akka] def systemActorOf(props: Props, name: String): ActorRef = { implicit val timeout = settings.CreationTimeout - Await.result(Futures.ask(systemGuardian, CreateChild(props, name)), timeout.duration) match { + askAndAwait(systemGuardian, CreateChild(props, name)) match { case ref: ActorRef ⇒ ref case ex: Exception ⇒ throw ex } @@ -297,7 +301,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor def actorOf(props: Props, name: String): ActorRef = { implicit val timeout = settings.CreationTimeout - Await.result(Futures.ask(guardian, CreateChild(props, name)), timeout.duration) match { + askAndAwait(guardian, CreateChild(props, name)) match { case ref: ActorRef ⇒ ref case ex: Exception ⇒ throw ex } @@ -305,7 +309,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor def actorOf(props: Props): ActorRef = { implicit val timeout = settings.CreationTimeout - Await.result(Futures.ask(guardian, CreateRandomNameChild(props)), timeout.duration) match { + askAndAwait(guardian, CreateRandomNameChild(props)) match { case ref: ActorRef ⇒ ref case ex: Exception ⇒ throw ex } @@ -317,8 +321,8 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor val guard = guardian.path val sys = systemGuardian.path path.parent match { - case `guard` ⇒ Await.result(Futures.ask(guardian, StopChild(actor)), timeout.duration) - case `sys` ⇒ Await.result(Futures.ask(systemGuardian, StopChild(actor)), timeout.duration) + case `guard` ⇒ askAndAwait(guardian, StopChild(actor)) + case `sys` ⇒ askAndAwait(systemGuardian, StopChild(actor)) case _ ⇒ actor.asInstanceOf[InternalActorRef].stop() } } diff --git a/akka-actor/src/main/scala/akka/actor/package.scala b/akka-actor/src/main/scala/akka/actor/package.scala index 0b06470c79..cedc7befd7 100644 --- a/akka-actor/src/main/scala/akka/actor/package.scala +++ b/akka-actor/src/main/scala/akka/actor/package.scala @@ -41,4 +41,26 @@ package object actor { } } + // Implicit for converting a Promise to an actor. + // Symmetric to the future2actor conversion, which allows + // piping a Future result (read side) to an Actor's mailbox, this + // conversion allows using an Actor to complete a Promise (write side) + // + // Future.ask / actor ? message is now a trivial implementation that can + // also be done in user code (assuming actorRef, timeout and dispatcher implicits): + // + // Future.ask(actor, message) = { + // val promise = Promise[Any]() + // actor ! (message, promise) + // promise + // } + + @inline implicit def promise2actor(promise: akka.dispatch.Promise[Any])(implicit actorRef: ActorRef, timeout: akka.util.Timeout) = { + val provider = actorRef.asInstanceOf[InternalActorRef].provider + provider.ask(promise, timeout) match { + case Some(ref) ⇒ ref + case None ⇒ null + } + } + } diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index fc4225600d..6eb408fd7a 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -55,14 +55,10 @@ object Await { object Futures { def ask(actor: ActorRef, message: Any)(implicit timeout: Timeout): Future[Any] = { - val provider = actor.asInstanceOf[InternalActorRef].provider - val promise = Promise[Any]()(provider.dispatcher) - provider.ask(promise, timeout) match { - case Some(a) ⇒ - actor.!(message)(a) - case None ⇒ - actor.!(message)(null) - } + implicit val dispatcher = actor.asInstanceOf[InternalActorRef].provider.dispatcher + implicit val actorRefContext = actor // for promise2actor implicit conversion + val promise = Promise[Any]() + actor.!(message)(promise) promise } diff --git a/akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java b/akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java index e6b45f675c..75fa92cd8f 100644 --- a/akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java +++ b/akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java @@ -10,6 +10,7 @@ import org.junit.Test; //#imports import akka.actor.*; import akka.dispatch.Await; +import static akka.dispatch.Futures.ask; import akka.transactor.Coordinated; import akka.util.Duration; import akka.util.Timeout; @@ -30,7 +31,7 @@ public class TransactorDocTest { counter1.tell(new Coordinated(new Increment(counter2), timeout)); - Integer count = (Integer) Await.result(counter1.ask("GetCount", timeout), timeout.duration()); + Integer count = (Integer) Await.result(ask(counter1, "GetCount", timeout), timeout.duration()); //#coordinated-example assertEquals(count, new Integer(1)); @@ -71,7 +72,7 @@ public class TransactorDocTest { counter.tell(coordinated.coordinate(new Increment())); coordinated.await(); - Integer count = (Integer) Await.result(counter.ask("GetCount", timeout), timeout.duration()); + Integer count = (Integer) Await.result(ask(counter, "GetCount", timeout), timeout.duration()); assertEquals(count, new Integer(1)); system.shutdown(); @@ -88,10 +89,10 @@ public class TransactorDocTest { friendlyCounter.tell(coordinated.coordinate(new Increment(friend))); coordinated.await(); - Integer count1 = (Integer) Await.result(friendlyCounter.ask("GetCount", timeout), timeout.duration()); + Integer count1 = (Integer) Await.result(ask(friendlyCounter, "GetCount", timeout), timeout.duration()); assertEquals(count1, new Integer(1)); - Integer count2 = (Integer) Await.result(friend.ask("GetCount", timeout), timeout.duration()); + Integer count2 = (Integer) Await.result(ask(friend, "GetCount", timeout), timeout.duration()); assertEquals(count2, new Integer(1)); system.shutdown(); diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java b/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java index 7fce881b2c..267cf261b7 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java @@ -18,6 +18,7 @@ import akka.actor.UntypedActor; import akka.actor.UntypedActorFactory; import akka.dispatch.Await; import akka.dispatch.Future; +import static akka.dispatch.Futures.ask; import akka.testkit.AkkaSpec; import akka.testkit.EventFilter; import akka.testkit.ErrorFilter; @@ -80,7 +81,7 @@ public class UntypedCoordinatedIncrementTest { } catch (InterruptedException exception) { } for (ActorRef counter : counters) { - Future future = counter.ask("GetCount", timeout); + Future future = ask(counter, "GetCount", timeout); int count = (Integer) Await.result(future, timeout.duration()); assertEquals(1, count); } @@ -102,7 +103,7 @@ public class UntypedCoordinatedIncrementTest { } catch (InterruptedException exception) { } for (ActorRef counter : counters) { - Futurefuture = counter.ask("GetCount", timeout); + Futurefuture = ask(counter, "GetCount", timeout); int count = (Integer) Await.result(future, timeout.duration()); assertEquals(0, count); } diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java b/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java index 9e2cf39f8d..3c80d659cf 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java @@ -18,6 +18,7 @@ import akka.actor.UntypedActor; import akka.actor.UntypedActorFactory; import akka.dispatch.Await; import akka.dispatch.Future; +import static akka.dispatch.Futures.ask; import akka.testkit.AkkaSpec; import akka.testkit.EventFilter; import akka.testkit.ErrorFilter; @@ -81,7 +82,7 @@ public class UntypedTransactorTest { } catch (InterruptedException exception) { } for (ActorRef counter : counters) { - Future future = counter.ask("GetCount", timeout); + Future future = ask(counter, "GetCount", timeout); int count = (Integer) Await.result(future, timeout.duration()); assertEquals(1, count); } @@ -103,7 +104,7 @@ public class UntypedTransactorTest { } catch (InterruptedException exception) { } for (ActorRef counter : counters) { - Future future = counter.ask("GetCount", timeout); + Future future = ask(counter, "GetCount", timeout); int count = (Integer) Await.result(future, timeout.duration()); assertEquals(0, count); } From a342bb93eae3bf203bee94e406909c53f5a5df66 Mon Sep 17 00:00:00 2001 From: Nikolay Botev Date: Sat, 31 Dec 2011 17:01:19 -0800 Subject: [PATCH 008/152] WIP: first compiling base --- .../scala/akka/actor/ActorRefProvider.scala | 2 +- .../main/scala/akka/actor/ActorSystem.scala | 15 +-- .../main/scala/akka/actor/TypedActor.scala | 7 +- .../src/main/scala/akka/actor/package.scala | 107 +++++++++++++++++- .../src/main/scala/akka/dispatch/Future.scala | 75 ------------ .../src/main/scala/akka/event/Logging.scala | 5 +- 6 files changed, 115 insertions(+), 96 deletions(-) mode change 100644 => 100755 akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala old mode 100644 new mode 100755 index 0b3215cd60..6af8c36198 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -520,7 +520,7 @@ class LocalActorRefProvider( // case other ⇒ result.success(other) // } //}), systemGuardian, systemGuardian.path / "promise" / tempName(), false, None) - //val ff = system.scheduler.scheduleOnce(t.duration) { b.stop() } + //val ff = system.scheduler.scheduleOnce(t.duration) { result.failure(new AskTimeoutException("Timed out")) } //result onComplete { _ ⇒ // b.stop() // ff.cancel() diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index e375f29e4b..098099c1d1 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -7,6 +7,7 @@ import akka.config.ConfigurationException import akka.actor._ import akka.event._ import akka.dispatch._ +import akka.patterns.ask import akka.util.duration._ import akka.util.Timeout import akka.util.Timeout._ @@ -287,13 +288,9 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor protected def systemImpl = this - @inline private def askAndAwait(actorRef: ActorRef, message: Any)(implicit timeout: akka.util.Timeout): Any = { - Await.result(Futures.ask(actorRef, message), timeout.duration) - } - private[akka] def systemActorOf(props: Props, name: String): ActorRef = { implicit val timeout = settings.CreationTimeout - askAndAwait(systemGuardian, CreateChild(props, name)) match { + Await.result(systemGuardian ? CreateChild(props, name), timeout.duration) match { case ref: ActorRef ⇒ ref case ex: Exception ⇒ throw ex } @@ -301,7 +298,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor def actorOf(props: Props, name: String): ActorRef = { implicit val timeout = settings.CreationTimeout - askAndAwait(guardian, CreateChild(props, name)) match { + Await.result(guardian ? CreateChild(props, name), timeout.duration) match { case ref: ActorRef ⇒ ref case ex: Exception ⇒ throw ex } @@ -309,7 +306,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor def actorOf(props: Props): ActorRef = { implicit val timeout = settings.CreationTimeout - askAndAwait(guardian, CreateRandomNameChild(props)) match { + Await.result(guardian ? CreateRandomNameChild(props), timeout.duration) match { case ref: ActorRef ⇒ ref case ex: Exception ⇒ throw ex } @@ -321,8 +318,8 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor val guard = guardian.path val sys = systemGuardian.path path.parent match { - case `guard` ⇒ askAndAwait(guardian, StopChild(actor)) - case `sys` ⇒ askAndAwait(systemGuardian, StopChild(actor)) + case `guard` ⇒ Await.result(guardian ? StopChild(actor), timeout.duration) + case `sys` ⇒ Await.result(systemGuardian ? StopChild(actor), timeout.duration) case _ ⇒ actor.asInstanceOf[InternalActorRef].stop() } } diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index 24ccda0f8d..87fbd98f47 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -396,17 +396,18 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi case "equals" ⇒ (args.length == 1 && (proxy eq args(0)) || actor == extension.getActorRefFor(args(0))).asInstanceOf[AnyRef] //Force boxing of the boolean case "hashCode" ⇒ actor.hashCode.asInstanceOf[AnyRef] case _ ⇒ + import akka.patterns.ask MethodCall(method, args) match { case m if m.isOneWay ⇒ actor ! m; null //Null return value - case m if m.returnsFuture_? ⇒ Futures.ask(actor, m)(timeout) + case m if m.returnsFuture_? ⇒ actor.?(m, timeout) case m if m.returnsJOption_? || m.returnsOption_? ⇒ - val f = Futures.ask(actor, m)(timeout) + val f = actor.?(m, timeout) (try { Await.ready(f, timeout.duration).value } catch { case _: TimeoutException ⇒ None }) match { case None | Some(Right(null)) ⇒ if (m.returnsJOption_?) JOption.none[Any] else None case Some(Right(joption: AnyRef)) ⇒ joption case Some(Left(ex)) ⇒ throw ex } - case m ⇒ Await.result(Futures.ask(actor, m)(timeout), timeout.duration).asInstanceOf[AnyRef] + case m ⇒ Await.result(actor.?(m, timeout), timeout.duration).asInstanceOf[AnyRef] } } } diff --git a/akka-actor/src/main/scala/akka/actor/package.scala b/akka-actor/src/main/scala/akka/actor/package.scala index cedc7befd7..f67a3bb1fa 100644 --- a/akka-actor/src/main/scala/akka/actor/package.scala +++ b/akka-actor/src/main/scala/akka/actor/package.scala @@ -8,9 +8,6 @@ package object actor { implicit def actorRef2Scala(ref: ActorRef): ScalaActorRef = ref.asInstanceOf[ScalaActorRef] implicit def scala2ActorRef(ref: ScalaActorRef): ActorRef = ref.asInstanceOf[ActorRef] - implicit def actorRef2Askable(actorRef: ActorRef) = new dispatch.AskableActorRef(actorRef) - implicit def askable2ActorRef(askable: dispatch.AskableActorRef) = askable.actorRef - type Uuid = com.eaio.uuid.UUID def newUuid(): Uuid = new Uuid() @@ -41,7 +38,17 @@ package object actor { } } - // Implicit for converting a Promise to an actor. +} + +package object patterns { + + import akka.actor.{ ActorRef, InternalActorRef } + import akka.dispatch.Promise + import akka.util.Timeout + + implicit def ask(actorRef: ActorRef): AskableActorRef = new AskableActorRef()(actorRef) + + // Implicit for converting a Promise to an ActorRef. // Symmetric to the future2actor conversion, which allows // piping a Future result (read side) to an Actor's mailbox, this // conversion allows using an Actor to complete a Promise (write side) @@ -49,13 +56,13 @@ package object actor { // Future.ask / actor ? message is now a trivial implementation that can // also be done in user code (assuming actorRef, timeout and dispatcher implicits): // - // Future.ask(actor, message) = { + // Patterns.ask(actor, message) = { // val promise = Promise[Any]() // actor ! (message, promise) // promise // } - @inline implicit def promise2actor(promise: akka.dispatch.Promise[Any])(implicit actorRef: ActorRef, timeout: akka.util.Timeout) = { + @inline implicit def promise2actorRef(promise: Promise[Any])(implicit actorRef: ActorRef, timeout: Timeout): ActorRef = { val provider = actorRef.asInstanceOf[InternalActorRef].provider provider.ask(promise, timeout) match { case Some(ref) ⇒ ref @@ -64,3 +71,91 @@ package object actor { } } + +package patterns { + + import akka.actor.{ ActorRef, InternalActorRef } + import akka.dispatch.{ Future, Promise } + import akka.util.Timeout + + final class AskableActorRef(implicit val actorRef: ActorRef) { + + /** + * Akka Java API. + * + * Sends a message asynchronously returns a future holding the eventual reply message. + * The Future will be completed with an [[akka.actor.AskTimeoutException]] after the given + * timeout has expired. + * + * NOTE: + * Use this method with care. In most cases it is better to use 'tell' together with the sender + * parameter to implement non-blocking request/response message exchanges. + * + * If you are sending messages using ask and using blocking operations on the Future, such as + * 'get', then you have to use getContext().sender().tell(...) + * in the target actor to send a reply message to the original sender, and thereby completing the Future, + * otherwise the sender will block until the timeout expires. + * + * When using future callbacks, inside actors you need to carefully avoid closing over + * the containing actor’s reference, i.e. do not call methods or access mutable state + * on the enclosing actor from within the callback. This would break the actor + * encapsulation and may introduce synchronization bugs and race conditions because + * the callback will be scheduled concurrently to the enclosing actor. Unfortunately + * there is not yet a way to detect these illegal accesses at compile time. + */ + def ask(message: AnyRef, timeout: Timeout): Future[AnyRef] = ?(message, timeout).asInstanceOf[Future[AnyRef]] + + def ask(message: AnyRef, timeoutMillis: Long): Future[AnyRef] = ask(message, new Timeout(timeoutMillis)) + + /** + * Sends a message asynchronously, returning a future which may eventually hold the reply. + * The Future will be completed with an [[akka.actor.AskTimeoutException]] after the given + * timeout has expired. + * + * NOTE: + * Use this method with care. In most cases it is better to use '!' together with implicit or explicit + * sender parameter to implement non-blocking request/response message exchanges. + * + * If you are sending messages using ask and using blocking operations on the Future, such as + * 'get', then you have to use getContext().sender().tell(...) + * in the target actor to send a reply message to the original sender, and thereby completing the Future, + * otherwise the sender will block until the timeout expires. + * + * When using future callbacks, inside actors you need to carefully avoid closing over + * the containing actor’s reference, i.e. do not call methods or access mutable state + * on the enclosing actor from within the callback. This would break the actor + * encapsulation and may introduce synchronization bugs and race conditions because + * the callback will be scheduled concurrently to the enclosing actor. Unfortunately + * there is not yet a way to detect these illegal accesses at compile time. + */ + def ?(message: Any)(implicit timeout: Timeout): Future[Any] = { + implicit val dispatcher = actorRef.asInstanceOf[InternalActorRef].provider.dispatcher + val promise = Promise[Any]() + actorRef.!(message)(promise) + promise + } + + /** + * Sends a message asynchronously, returning a future which may eventually hold the reply. + * The implicit parameter with the default value is just there to disambiguate it from the version that takes the + * implicit timeout + */ + def ?(message: Any, timeout: Timeout)(implicit ignore: Int = 0): Future[Any] = ?(message)(timeout) + } + +} + +object Patterns { + + import akka.actor.ActorRef + import akka.dispatch.Future + import akka.patterns.{ ask => actorRef2Askable } + import akka.util.Timeout + + def ask(actor: ActorRef, message: Any, timeout: Timeout): Future[Any] = + actorRef2Askable(actor).?(message)(timeout) + + def ask(actor: ActorRef, message: Any, timeoutMillis: Long): Future[Any] = + actorRef2Askable(actor).?(message)(new Timeout(timeoutMillis)) + +} diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 6eb408fd7a..fea97fbaf3 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -24,7 +24,6 @@ import java.util.concurrent.atomic.{ AtomicReferenceFieldUpdater, AtomicInteger, import akka.dispatch.Await.CanAwait import java.util.concurrent._ import akka.actor.ActorSystem -import akka.actor.{ ActorRef, InternalActorRef } object Await { sealed trait CanAwait @@ -54,20 +53,6 @@ object Await { */ object Futures { - def ask(actor: ActorRef, message: Any)(implicit timeout: Timeout): Future[Any] = { - implicit val dispatcher = actor.asInstanceOf[InternalActorRef].provider.dispatcher - implicit val actorRefContext = actor // for promise2actor implicit conversion - val promise = Promise[Any]() - actor.!(message)(promise) - promise - } - - def ask(actor: ActorRef, message: Any, timeout: Timeout)(implicit ignore: Int = 0): Future[Any] = - ask(actor, message)(timeout) - - def ask(actor: ActorRef, message: Any, timeoutMillis: Long): Future[Any] = - ask(actor, message)(new Timeout(timeoutMillis)) - /** * Java API, equivalent to Future.apply */ @@ -149,66 +134,6 @@ object Futures { } } -final class AskableActorRef(val actorRef: ActorRef) { - - /** - * Akka Java API. - * - * Sends a message asynchronously returns a future holding the eventual reply message. - * The Future will be completed with an [[akka.actor.AskTimeoutException]] after the given - * timeout has expired. - * - * NOTE: - * Use this method with care. In most cases it is better to use 'tell' together with the sender - * parameter to implement non-blocking request/response message exchanges. - * - * If you are sending messages using ask and using blocking operations on the Future, such as - * 'get', then you have to use getContext().sender().tell(...) - * in the target actor to send a reply message to the original sender, and thereby completing the Future, - * otherwise the sender will block until the timeout expires. - * - * When using future callbacks, inside actors you need to carefully avoid closing over - * the containing actor’s reference, i.e. do not call methods or access mutable state - * on the enclosing actor from within the callback. This would break the actor - * encapsulation and may introduce synchronization bugs and race conditions because - * the callback will be scheduled concurrently to the enclosing actor. Unfortunately - * there is not yet a way to detect these illegal accesses at compile time. - */ - def ask(message: AnyRef, timeout: Timeout): Future[AnyRef] = ?(message, timeout).asInstanceOf[Future[AnyRef]] - - def ask(message: AnyRef, timeoutMillis: Long): Future[AnyRef] = ask(message, new Timeout(timeoutMillis)) - - /** - * Sends a message asynchronously, returning a future which may eventually hold the reply. - * The Future will be completed with an [[akka.actor.AskTimeoutException]] after the given - * timeout has expired. - * - * NOTE: - * Use this method with care. In most cases it is better to use '!' together with implicit or explicit - * sender parameter to implement non-blocking request/response message exchanges. - * - * If you are sending messages using ask and using blocking operations on the Future, such as - * 'get', then you have to use getContext().sender().tell(...) - * in the target actor to send a reply message to the original sender, and thereby completing the Future, - * otherwise the sender will block until the timeout expires. - * - * When using future callbacks, inside actors you need to carefully avoid closing over - * the containing actor’s reference, i.e. do not call methods or access mutable state - * on the enclosing actor from within the callback. This would break the actor - * encapsulation and may introduce synchronization bugs and race conditions because - * the callback will be scheduled concurrently to the enclosing actor. Unfortunately - * there is not yet a way to detect these illegal accesses at compile time. - */ - def ?(message: Any)(implicit timeout: Timeout): Future[Any] = Futures.ask(actorRef, message) - - /** - * Sends a message asynchronously, returning a future which may eventually hold the reply. - * The implicit parameter with the default value is just there to disambiguate it from the version that takes the - * implicit timeout - */ - def ?(message: Any, timeout: Timeout)(implicit ignore: Int = 0): Future[Any] = ?(message)(timeout) -} - object Future { /** diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index 4099dc06b5..bb0f881c94 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -14,7 +14,7 @@ import akka.util.Timeout import java.util.concurrent.atomic.AtomicInteger import scala.util.control.NoStackTrace import java.util.concurrent.TimeoutException -import akka.dispatch.{ Await, Futures } +import akka.dispatch.Await object LoggingBus { implicit def fromActorSystem(system: ActorSystem): LoggingBus = system.eventStream @@ -158,7 +158,8 @@ trait LoggingBus extends ActorEventBus { val name = "log" + Extension(system).id() + "-" + simpleName(clazz) val actor = system.systemActorOf(Props(clazz), name) implicit val timeout = Timeout(3 seconds) - val response = try Await.result(Futures.ask(actor, InitializeLogger(this)), timeout.duration) catch { + import akka.patterns.ask + val response = try Await.result(actor ? InitializeLogger(this), timeout.duration) catch { case _: TimeoutException ⇒ publish(Warning(simpleName(this), "Logger " + name + " did not respond within " + timeout + " to InitializeLogger(bus)")) } From ce1d2f4721a941b49c61f711719a20ccc864ee72 Mon Sep 17 00:00:00 2001 From: Nikolay Botev Date: Sat, 31 Dec 2011 17:42:13 -0800 Subject: [PATCH 009/152] akka.patterns.ask everywhere --- .../ActorFireForgetRequestReplySpec.scala | 1 + .../scala/akka/actor/ActorLifeCycleSpec.scala | 1 + .../scala/akka/actor/ActorLookupSpec.scala | 1 + .../test/scala/akka/actor/ActorRefSpec.scala | 1 + .../scala/akka/actor/ActorTimeoutSpec.scala | 1 + .../scala/akka/actor/DeathWatchSpec.scala | 1 + .../scala/akka/actor/ForwardActorSpec.scala | 1 + .../src/test/scala/akka/actor/IOActor.scala | 1 + .../akka/actor/RestartStrategySpec.scala | 1 + .../test/scala/akka/actor/SchedulerSpec.scala | 1 + .../akka/actor/SupervisorHierarchySpec.scala | 1 + .../scala/akka/actor/SupervisorMiscSpec.scala | 1 + .../scala/akka/actor/SupervisorSpec.scala | 1 + .../scala/akka/actor/SupervisorTreeSpec.scala | 1 + .../test/scala/akka/actor/Ticket669Spec.scala | 1 + .../scala/akka/actor/TypedActorSpec.scala | 1 + .../akka/actor/dispatch/ActorModelSpec.scala | 1 + .../actor/dispatch/DispatcherActorSpec.scala | 1 + .../akka/actor/dispatch/PinnedActorSpec.scala | 1 + .../scala/akka/dataflow/Future2Actor.scala | 1 + .../test/scala/akka/dispatch/FutureSpec.scala | 1 + .../dispatch/PriorityDispatcherSpec.scala | 2 +- .../scala/akka/routing/ActorPoolSpec.scala | 1 + .../routing/ConfiguredLocalRoutingSpec.scala | 1 + .../test/scala/akka/routing/RoutingSpec.scala | 1 + .../akka/serialization/SerializeSpec.scala | 1 + .../scala/akka/ticket/Ticket703Spec.scala | 1 + .../src/main/scala/akka/actor/package.scala | 7 ++++-- .../src/main/scala/akka/agent/Agent.scala | 1 + .../docs/actor/FaultHandlingTestBase.java | 2 +- .../docs/actor/UntypedActorDocTestBase.java | 5 +++-- .../akka/docs/future/FutureDocTestBase.java | 3 ++- .../docs/transactor/TransactorDocTest.java | 2 +- .../code/akka/docs/actor/ActorDocSpec.scala | 8 ++++--- .../code/akka/docs/future/FutureDocSpec.scala | 22 +++++++++---------- .../akka/docs/routing/RouterTypeExample.scala | 5 +++-- .../akka/docs/testkit/TestkitDocSpec.scala | 4 +++- .../docs/transactor/TransactorDocSpec.scala | 1 + .../src/main/scala/akka/remote/Gossiper.scala | 1 + .../DirectRoutedRemoteActorMultiJvmSpec.scala | 1 + .../remote/NewRemoteActorMultiJvmSpec.scala | 1 + .../RandomRoutedRemoteActorMultiJvmSpec.scala | 1 + ...ndRobinRoutedRemoteActorMultiJvmSpec.scala | 1 + .../akka/remote/RemoteCommunicationSpec.scala | 1 + .../scala/akka/testkit/TestActorRef.scala | 1 + .../test/scala/akka/testkit/AkkaSpec.scala | 1 + .../scala/akka/testkit/TestActorRefSpec.scala | 1 + .../scala/akka/testkit/TestProbeSpec.scala | 1 + .../UntypedCoordinatedIncrementTest.java | 2 +- .../transactor/UntypedTransactorTest.java | 2 +- .../transactor/CoordinatedIncrementSpec.scala | 1 + .../akka/transactor/FickleFriendsSpec.scala | 1 + .../akka/transactor/TransactorSpec.scala | 1 + 53 files changed, 78 insertions(+), 27 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala index c059497259..8a4d643e29 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala @@ -8,6 +8,7 @@ import akka.testkit._ import org.scalatest.BeforeAndAfterEach import akka.util.duration._ import akka.dispatch.Await +import akka.patterns.ask object ActorFireForgetRequestReplySpec { diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala index b203ff256f..016b738c45 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala @@ -12,6 +12,7 @@ import akka.testkit._ import akka.util.duration._ import java.util.concurrent.atomic._ import akka.dispatch.Await +import akka.patterns.ask object ActorLifeCycleSpec { diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala index da6115ef5f..8a990e18ca 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala @@ -6,6 +6,7 @@ package akka.actor import akka.testkit._ import akka.util.duration._ import akka.dispatch.Await +import akka.patterns.ask object ActorLookupSpec { diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala index ceea20b2c7..a78a99a79b 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala @@ -15,6 +15,7 @@ import akka.util.ReflectiveAccess import akka.serialization.Serialization import java.util.concurrent.{ CountDownLatch, TimeUnit } import akka.dispatch.{ Await, DefaultPromise, Promise, Future } +import akka.patterns.ask object ActorRefSpec { diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala index 038e3fc9f1..1c9a7e67d6 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala @@ -10,6 +10,7 @@ import akka.testkit.DefaultTimeout import java.util.concurrent.TimeoutException import akka.dispatch.Await import akka.util.Timeout +import akka.patterns.ask @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class ActorTimeoutSpec extends AkkaSpec with BeforeAndAfterAll with DefaultTimeout { diff --git a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala index 30828c1014..1a9c84ecd9 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala @@ -8,6 +8,7 @@ import akka.testkit._ import akka.util.duration._ import java.util.concurrent.atomic._ import akka.dispatch.Await +import akka.patterns.ask @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class LocalDeathWatchSpec extends AkkaSpec with ImplicitSender with DefaultTimeout with DeathWatchSpec diff --git a/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala index 12b0c796f6..10df9d2218 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala @@ -9,6 +9,7 @@ import akka.util.duration._ import Actor._ import akka.util.Duration import akka.dispatch.Await +import akka.patterns.ask object ForwardActorSpec { val ExpectedMessage = "FOO" diff --git a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala index 757acb1fd0..aaa86e9c3e 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala @@ -11,6 +11,7 @@ import akka.util.cps._ import scala.util.continuations._ import akka.testkit._ import akka.dispatch.{ Await, Future } +import akka.patterns.ask object IOActorSpec { import IO._ diff --git a/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala index b627046052..0730b8de7c 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala @@ -14,6 +14,7 @@ import akka.testkit.AkkaSpec import akka.testkit.DefaultTimeout import akka.testkit.TestLatch import akka.util.duration._ +import akka.patterns.ask @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class RestartStrategySpec extends AkkaSpec with DefaultTimeout { diff --git a/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala index ba06a90023..1269558cf9 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala @@ -8,6 +8,7 @@ import java.util.concurrent.{ CountDownLatch, ConcurrentLinkedQueue, TimeUnit } import akka.testkit.DefaultTimeout import akka.testkit.TestLatch import akka.dispatch.Await +import akka.patterns.ask @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout { diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala index fb34e2345b..88139f97f8 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala @@ -8,6 +8,7 @@ import akka.testkit._ import java.util.concurrent.{ TimeUnit, CountDownLatch } import akka.dispatch.Await +import akka.patterns.ask object SupervisorHierarchySpec { class FireWorkerException(msg: String) extends Exception(msg) diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala index 653342c193..8b32f76e49 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala @@ -8,6 +8,7 @@ import akka.dispatch.{ PinnedDispatcher, Dispatchers, Await } import java.util.concurrent.{ TimeUnit, CountDownLatch } import akka.testkit.AkkaSpec import akka.testkit.DefaultTimeout +import akka.patterns.ask object SupervisorMiscSpec { val config = """ diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala index e68e6f3906..8405e29e8e 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala @@ -11,6 +11,7 @@ import akka.testkit.TestEvent._ import akka.testkit._ import java.util.concurrent.atomic.AtomicInteger import akka.dispatch.Await +import akka.patterns.ask object SupervisorSpec { val Timeout = 5 seconds diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala index 3985f6ea48..6ebca690a7 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala @@ -12,6 +12,7 @@ import akka.testkit.AkkaSpec import akka.testkit.ImplicitSender import akka.testkit.DefaultTimeout import akka.dispatch.{ Await, Dispatchers } +import akka.patterns.ask @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class SupervisorTreeSpec extends AkkaSpec with ImplicitSender with DefaultTimeout { diff --git a/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala b/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala index 3b5f29c950..f3a0784da2 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala @@ -11,6 +11,7 @@ import akka.testkit.AkkaSpec import akka.testkit.ImplicitSender import akka.testkit.DefaultTimeout import akka.dispatch.Await +import akka.patterns.ask @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class Ticket669Spec extends AkkaSpec with BeforeAndAfterAll with ImplicitSender with DefaultTimeout { diff --git a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala index 6a6500b131..7cde57720c 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala @@ -18,6 +18,7 @@ import java.util.concurrent.{ TimeUnit, CountDownLatch } import akka.japi.{ Creator, Option ⇒ JOption } import akka.testkit.DefaultTimeout import akka.dispatch.{ Await, Dispatchers, Future, Promise } +import akka.patterns.ask object TypedActorSpec { diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala index 9debbd053c..c242617753 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala @@ -20,6 +20,7 @@ import akka.util.duration._ import akka.event.Logging.Error import com.typesafe.config.Config import akka.util.Duration +import akka.patterns.ask object ActorModelSpec { diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala index d75bad30c6..1160883e09 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala @@ -8,6 +8,7 @@ import akka.util.Duration import akka.util.duration._ import akka.testkit.DefaultTimeout import akka.dispatch.{ Await, PinnedDispatcher, Dispatchers, Dispatcher } +import akka.patterns.ask object DispatcherActorSpec { val config = """ diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala index 6ac18f9947..4ddcb8ba12 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala @@ -7,6 +7,7 @@ import akka.actor.{ Props, Actor } import akka.testkit.AkkaSpec import org.scalatest.BeforeAndAfterEach import akka.dispatch.{ Await, PinnedDispatcher, Dispatchers } +import akka.patterns.ask object PinnedActorSpec { val config = """ diff --git a/akka-actor-tests/src/test/scala/akka/dataflow/Future2Actor.scala b/akka-actor-tests/src/test/scala/akka/dataflow/Future2Actor.scala index 5d24b9678f..48992a5ee7 100644 --- a/akka-actor-tests/src/test/scala/akka/dataflow/Future2Actor.scala +++ b/akka-actor-tests/src/test/scala/akka/dataflow/Future2Actor.scala @@ -9,6 +9,7 @@ import akka.actor.future2actor import akka.util.duration._ import akka.testkit.AkkaSpec import akka.testkit.DefaultTimeout +import akka.patterns.ask class Future2ActorSpec extends AkkaSpec with DefaultTimeout { diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala index 4fbb67fbb4..c874ea68b7 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala @@ -16,6 +16,7 @@ import akka.testkit.DefaultTimeout import akka.testkit.TestLatch import java.util.concurrent.{ TimeoutException, TimeUnit, CountDownLatch } import scala.runtime.NonLocalReturnControl +import akka.patterns.ask object FutureSpec { class TestActor extends Actor { diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala index ec6aab48be..82fcc372fa 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala @@ -2,7 +2,7 @@ package akka.dispatch import akka.actor.{ Props, LocalActorRef, Actor } import akka.testkit.AkkaSpec -import akka.util.Duration +import akka.patterns.ask import akka.util.duration._ import akka.testkit.DefaultTimeout import com.typesafe.config.Config diff --git a/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala index f18fd2e5e1..bfdb168156 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala @@ -6,6 +6,7 @@ import akka.util.duration._ import java.util.concurrent.atomic.{ AtomicBoolean, AtomicInteger } import akka.testkit.AkkaSpec import akka.dispatch.{ Await, Promise, Future } +import akka.patterns.ask object ActorPoolSpec { diff --git a/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala index dd4e45f5cb..2d4a4be1c2 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala @@ -6,6 +6,7 @@ import java.util.concurrent.atomic.AtomicInteger import akka.testkit._ import akka.util.duration._ import akka.dispatch.Await +import akka.patterns.ask @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class ConfiguredLocalRoutingSpec extends AkkaSpec with DefaultTimeout with ImplicitSender { diff --git a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala index fb2aa50372..dff790f50f 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala @@ -12,6 +12,7 @@ import akka.dispatch.Await import akka.util.Duration import akka.config.ConfigurationException import com.typesafe.config.ConfigFactory +import akka.patterns.ask object RoutingSpec { diff --git a/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala b/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala index 51dc26d6c5..cfec55413d 100644 --- a/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala @@ -13,6 +13,7 @@ import akka.util.Timeout import akka.util.duration._ import scala.reflect.BeanInfo import com.google.protobuf.Message +import akka.patterns.ask class ProtobufSerializer extends Serializer { val ARRAY_OF_BYTE_ARRAY = Array[Class[_]](classOf[Array[Byte]]) diff --git a/akka-actor-tests/src/test/scala/akka/ticket/Ticket703Spec.scala b/akka-actor-tests/src/test/scala/akka/ticket/Ticket703Spec.scala index f51beb7617..63367ff508 100644 --- a/akka-actor-tests/src/test/scala/akka/ticket/Ticket703Spec.scala +++ b/akka-actor-tests/src/test/scala/akka/ticket/Ticket703Spec.scala @@ -5,6 +5,7 @@ import akka.routing._ import akka.testkit.AkkaSpec import akka.dispatch.Await import akka.util.duration._ +import akka.patterns.ask @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class Ticket703Spec extends AkkaSpec { diff --git a/akka-actor/src/main/scala/akka/actor/package.scala b/akka-actor/src/main/scala/akka/actor/package.scala index f67a3bb1fa..acfbb19e5d 100644 --- a/akka-actor/src/main/scala/akka/actor/package.scala +++ b/akka-actor/src/main/scala/akka/actor/package.scala @@ -149,10 +149,13 @@ object Patterns { import akka.actor.ActorRef import akka.dispatch.Future - import akka.patterns.{ ask => actorRef2Askable } + import akka.patterns.{ ask ⇒ actorRef2Askable } import akka.util.Timeout - def ask(actor: ActorRef, message: Any, timeout: Timeout): Future[Any] = + def ask(actor: ActorRef, message: Any)(implicit timeout: Timeout): Future[Any] = + actorRef2Askable(actor).?(message) + + def ask(actor: ActorRef, message: Any, timeout: Timeout)(implicit ignore: Int = 0): Future[Any] = actorRef2Askable(actor).?(message)(timeout) def ask(actor: ActorRef, message: Any, timeoutMillis: Long): Future[Any] = diff --git a/akka-agent/src/main/scala/akka/agent/Agent.scala b/akka-agent/src/main/scala/akka/agent/Agent.scala index dffd8df1cc..8337c0f64b 100644 --- a/akka-agent/src/main/scala/akka/agent/Agent.scala +++ b/akka-agent/src/main/scala/akka/agent/Agent.scala @@ -7,6 +7,7 @@ package akka.agent import akka.actor._ import akka.japi.{ Function ⇒ JFunc, Procedure ⇒ JProc } import akka.dispatch._ +import akka.patterns.ask import akka.util.Timeout import scala.concurrent.stm._ diff --git a/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java b/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java index 1bc3b40c4b..f67347b2af 100644 --- a/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java +++ b/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java @@ -13,7 +13,7 @@ import akka.actor.Props; import akka.actor.Terminated; import akka.actor.UntypedActor; import akka.dispatch.Await; -import static akka.dispatch.Futures.ask; +import static akka.Patterns.ask; import akka.util.Duration; import akka.testkit.AkkaSpec; import akka.testkit.TestProbe; diff --git a/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java b/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java index be164aa850..4944f817f5 100644 --- a/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java +++ b/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java @@ -38,6 +38,7 @@ import org.junit.Test; import scala.Option; import java.lang.Object; import java.util.concurrent.TimeUnit; +import akka.Patterns; import static org.junit.Assert.*; @@ -118,7 +119,7 @@ public class UntypedActorDocTestBase { }), "myactor"); //#using-ask - Future future = Futures.ask(myActor, "Hello", 1000); + Future future = Patterns.ask(myActor, "Hello", 1000); Object result = Await.result(future, Duration.create(1, TimeUnit.SECONDS)); //#using-ask system.shutdown(); @@ -170,7 +171,7 @@ public class UntypedActorDocTestBase { public void useWatch() { ActorSystem system = ActorSystem.create("MySystem"); ActorRef myActor = system.actorOf(new Props(WatchActor.class)); - Future future = myActor.ask("kill", 1000); + Future future = Patterns.ask(myActor, "kill", 1000); assert Await.result(future, Duration.parse("1 second")).equals("finished"); system.shutdown(); } diff --git a/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java b/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java index 956e72eab9..dd10afd165 100644 --- a/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java +++ b/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java @@ -55,6 +55,7 @@ import akka.actor.UntypedActor; import akka.actor.ActorRef; import akka.actor.Props; import akka.dispatch.Futures; +import akka.Patterns; import static org.junit.Assert.*; @@ -78,7 +79,7 @@ public class FutureDocTestBase { String msg = "hello"; //#ask-blocking Timeout timeout = system.settings().ActorTimeout(); - Future future = Futures.ask(actor, msg, timeout); + Future future = Patterns.ask(actor, msg, timeout); String result = (String) Await.result(future, timeout.duration()); //#ask-blocking assertEquals("HELLO", result); diff --git a/akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java b/akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java index 75fa92cd8f..bb2f069dd0 100644 --- a/akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java +++ b/akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java @@ -10,7 +10,7 @@ import org.junit.Test; //#imports import akka.actor.*; import akka.dispatch.Await; -import static akka.dispatch.Futures.ask; +import static akka.Patterns.ask; import akka.transactor.Coordinated; import akka.util.Duration; import akka.util.Timeout; diff --git a/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala b/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala index 3907e6cf47..f804e29bac 100644 --- a/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala @@ -8,7 +8,7 @@ import akka.actor.Actor import akka.actor.Props import akka.event.Logging import akka.dispatch.Future -import akka.dispatch.Futures +import akka.Patterns //#imports1 @@ -230,10 +230,10 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) { val myActor = system.actorOf(Props(new MyActor), name = "myactor") implicit val timeout = system.settings.ActorTimeout - val future = Futures.ask(myActor, "hello") + val future = Patterns.ask(myActor, "hello") for (x ← future) println(x) //Prints "hello" - val result: Future[Int] = for (x ← Futures.ask(myActor, 3).mapTo[Int]) yield { 2 * x } + val result: Future[Int] = for (x ← Patterns.ask(myActor, 3).mapTo[Int]) yield { 2 * x } //#using-ask system.stop(myActor) @@ -244,6 +244,7 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) { //#using-implicit-timeout import akka.util.duration._ import akka.util.Timeout + import akka.patterns.ask implicit val timeout = Timeout(500 millis) val future = myActor ? "hello" //#using-implicit-timeout @@ -255,6 +256,7 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) { val myActor = system.actorOf(Props(new FirstActor)) //#using-explicit-timeout import akka.util.duration._ + import akka.patterns.ask val future = myActor ? ("hello", timeout = 500 millis) //#using-explicit-timeout Await.result(future, 500 millis) must be("hello") diff --git a/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala b/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala index bb96a4e3a0..f65f8224db 100644 --- a/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala @@ -10,10 +10,10 @@ import akka.actor.Actor import akka.actor.Props import akka.actor.Status.Failure import akka.dispatch.Future -import akka.dispatch.Futures import akka.dispatch.Await import akka.util.duration._ import akka.dispatch.Promise +import akka.Patterns object FutureDocSpec { @@ -47,7 +47,7 @@ class FutureDocSpec extends AkkaSpec { import akka.dispatch.Await implicit val timeout = system.settings.ActorTimeout - val future = Futures.ask(actor, msg) + val future = Patterns.ask(actor, msg) val result = Await.result(future, timeout.duration).asInstanceOf[String] //#ask-blocking result must be("HELLO") @@ -60,7 +60,7 @@ class FutureDocSpec extends AkkaSpec { //#map-to import akka.dispatch.Future - val future: Future[String] = Futures.ask(actor, msg).mapTo[String] + val future: Future[String] = Patterns.ask(actor, msg).mapTo[String] //#map-to Await.result(future, timeout.duration) must be("HELLO") } @@ -150,13 +150,13 @@ class FutureDocSpec extends AkkaSpec { import akka.dispatch.Await //#composing-wrong - val f1 = Futures.ask(actor1, msg1) - val f2 = Futures.ask(actor2, msg2) + val f1 = Patterns.ask(actor1, msg1) + val f2 = Patterns.ask(actor2, msg2) val a = Await.result(f1, 1 second).asInstanceOf[Int] val b = Await.result(f2, 1 second).asInstanceOf[Int] - val f3 = Futures.ask(actor3, (a + b)) + val f3 = Patterns.ask(actor3, (a + b)) val result = Await.result(f3, 1 second).asInstanceOf[Int] //#composing-wrong @@ -173,13 +173,13 @@ class FutureDocSpec extends AkkaSpec { import akka.dispatch.Await //#composing - val f1 = Futures.ask(actor1, msg1) - val f2 = Futures.ask(actor2, msg2) + val f1 = Patterns.ask(actor1, msg1) + val f2 = Patterns.ask(actor2, msg2) val f3 = for { a ← f1.mapTo[Int] b ← f2.mapTo[Int] - c ← Futures.ask(actor3, (a + b)).mapTo[Int] + c ← Patterns.ask(actor3, (a + b)).mapTo[Int] } yield c val result = Await.result(f3, 1 second).asInstanceOf[Int] @@ -192,7 +192,7 @@ class FutureDocSpec extends AkkaSpec { val oddActor = system.actorOf(Props[OddActor]) //#sequence-ask // oddActor returns odd numbers sequentially from 1 as a List[Future[Int]] - val listOfFutures = List.fill(100)(Futures.ask(oddActor, GetNext).mapTo[Int]) + val listOfFutures = List.fill(100)(Patterns.ask(oddActor, GetNext).mapTo[Int]) // now we have a Future[List[Int]] val futureList = Future.sequence(listOfFutures) @@ -240,7 +240,7 @@ class FutureDocSpec extends AkkaSpec { val actor = system.actorOf(Props[MyActor]) val msg1 = -1 //#recover - val future = Futures.ask(actor, msg1) recover { + val future = Patterns.ask(actor, msg1) recover { case e: ArithmeticException ⇒ 0 } //#recover diff --git a/akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala b/akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala index c366c011c1..bc3f370f5b 100644 --- a/akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala +++ b/akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala @@ -7,7 +7,8 @@ import akka.routing.{ ScatterGatherFirstCompletedRouter, BroadcastRouter, Random import annotation.tailrec import akka.actor.{ Props, Actor } import akka.util.duration._ -import akka.dispatch.{ Futures, Await } +import akka.dispatch.Await +import akka.patterns.ask case class FibonacciNumber(nbr: Int) @@ -71,7 +72,7 @@ class ParentActor extends Actor { Props[FibonacciActor].withRouter(ScatterGatherFirstCompletedRouter(within = 2 seconds)), "router") implicit val timeout = context.system.settings.ActorTimeout - val futureResult = Futures.ask(scatterGatherFirstCompletedRouter, FibonacciNumber(10)) + val futureResult = scatterGatherFirstCompletedRouter ? FibonacciNumber(10) val result = Await.result(futureResult, timeout.duration) //#scatterGatherFirstCompletedRouter println("The result of calculating Fibonacci for 10 is %d".format(result)) diff --git a/akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala b/akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala index c646f9ea54..b3ff8d2e05 100644 --- a/akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala @@ -120,6 +120,7 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender { import akka.testkit.TestActorRef import akka.util.duration._ import akka.dispatch.Await + import akka.patterns.ask val actorRef = TestActorRef(new MyActor) // hypothetical message stimulating a '42' answer @@ -203,9 +204,10 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender { "demonstrate probe reply" in { import akka.testkit.TestProbe import akka.util.duration._ + import akka.patterns.ask //#test-probe-reply val probe = TestProbe() - val future = Futures.ask(probe.ref, "hello") + val future = probe.ref ? "hello" probe.expectMsg(0 millis, "hello") // TestActor runs on CallingThreadDispatcher probe.sender ! "world" assert(future.isCompleted && future.value == Some(Right("world"))) diff --git a/akka-docs/scala/code/akka/docs/transactor/TransactorDocSpec.scala b/akka-docs/scala/code/akka/docs/transactor/TransactorDocSpec.scala index 246ea0a352..2cde9c9890 100644 --- a/akka-docs/scala/code/akka/docs/transactor/TransactorDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/transactor/TransactorDocSpec.scala @@ -141,6 +141,7 @@ class TransactorDocSpec extends AkkaSpec { import akka.dispatch.Await import akka.util.duration._ import akka.util.Timeout + import akka.patterns.ask val system = ActorSystem("app") diff --git a/akka-remote/src/main/scala/akka/remote/Gossiper.scala b/akka-remote/src/main/scala/akka/remote/Gossiper.scala index a12e5ecab1..3ff54b5efe 100644 --- a/akka-remote/src/main/scala/akka/remote/Gossiper.scala +++ b/akka-remote/src/main/scala/akka/remote/Gossiper.scala @@ -22,6 +22,7 @@ import scala.annotation.tailrec import com.google.protobuf.ByteString import java.util.concurrent.TimeoutException import akka.dispatch.Await +import akka.patterns.ask /** * Interface for node membership change listener. diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala index 1b7a561cda..58c52d8971 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala @@ -5,6 +5,7 @@ import akka.routing._ import akka.actor.{ Actor, Props } import akka.testkit._ import akka.dispatch.Await +import akka.patterns.ask object DirectRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { override def NrOfNodes = 2 diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala index f7a02c3988..cc8f62552c 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala @@ -6,6 +6,7 @@ import akka.routing._ import akka.testkit._ import akka.util.duration._ import akka.dispatch.Await +import akka.patterns.ask object NewRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { override def NrOfNodes = 2 diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala index b9440d28b3..d466c014b3 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala @@ -5,6 +5,7 @@ import akka.remote._ import akka.routing._ import akka.testkit.DefaultTimeout import akka.dispatch.Await +import akka.patterns.ask object RandomRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { override def NrOfNodes = 4 diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala index 5b5c6cbf6d..22eb8cd8c3 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala @@ -5,6 +5,7 @@ import akka.remote._ import akka.routing._ import akka.testkit.DefaultTimeout import akka.dispatch.Await +import akka.patterns.ask object RoundRobinRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { override def NrOfNodes = 4 diff --git a/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala index 0be7c0f361..17068cc351 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala @@ -7,6 +7,7 @@ import akka.testkit._ import akka.actor._ import com.typesafe.config._ import akka.dispatch.Await +import akka.patterns.ask object RemoteCommunicationSpec { class Echo extends Actor { diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala index 2d52f21226..01ce145afa 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala @@ -13,6 +13,7 @@ import java.util.concurrent.atomic.AtomicLong import akka.event.EventStream import scala.collection.immutable.Stack import akka.dispatch._ +import akka.patterns.ask /** * This special ActorRef is exclusively for use during unit testing in a single-threaded environment. Therefore, it diff --git a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala index b98937b126..e47587fc9c 100644 --- a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala @@ -17,6 +17,7 @@ import akka.actor.DeadLetter import java.util.concurrent.TimeoutException import akka.dispatch.{ Await, MessageDispatcher } import akka.dispatch.Dispatchers +import akka.patterns.ask object TimingTest extends Tag("timing") diff --git a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala index 747a9c90e9..b742165f1d 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala @@ -10,6 +10,7 @@ import akka.event.Logging.Warning import akka.dispatch.{ Future, Promise, Await } import akka.util.duration._ import akka.actor.ActorSystem +import akka.patterns.ask /** * Test whether TestActorRef behaves as an ActorRef should, besides its own spec. diff --git a/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala index 4723070299..6a4a9db420 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala @@ -6,6 +6,7 @@ import org.scalatest.{ BeforeAndAfterEach, WordSpec } import akka.actor._ import akka.util.duration._ import akka.dispatch.{ Await, Future } +import akka.patterns.ask @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class TestProbeSpec extends AkkaSpec with DefaultTimeout { diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java b/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java index 267cf261b7..d282a638fc 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java @@ -18,7 +18,7 @@ import akka.actor.UntypedActor; import akka.actor.UntypedActorFactory; import akka.dispatch.Await; import akka.dispatch.Future; -import static akka.dispatch.Futures.ask; +import static akka.Patterns.ask; import akka.testkit.AkkaSpec; import akka.testkit.EventFilter; import akka.testkit.ErrorFilter; diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java b/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java index 3c80d659cf..2a6e30ea81 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java @@ -18,7 +18,7 @@ import akka.actor.UntypedActor; import akka.actor.UntypedActorFactory; import akka.dispatch.Await; import akka.dispatch.Future; -import static akka.dispatch.Futures.ask; +import static akka.Patterns.ask; import akka.testkit.AkkaSpec; import akka.testkit.EventFilter; import akka.testkit.ErrorFilter; diff --git a/akka-transactor/src/test/scala/akka/transactor/CoordinatedIncrementSpec.scala b/akka-transactor/src/test/scala/akka/transactor/CoordinatedIncrementSpec.scala index 47067d3595..2c2e4376e9 100644 --- a/akka-transactor/src/test/scala/akka/transactor/CoordinatedIncrementSpec.scala +++ b/akka-transactor/src/test/scala/akka/transactor/CoordinatedIncrementSpec.scala @@ -12,6 +12,7 @@ import akka.util.duration._ import akka.util.Timeout import akka.testkit._ import scala.concurrent.stm._ +import akka.patterns.ask object CoordinatedIncrement { case class Increment(friends: Seq[ActorRef]) diff --git a/akka-transactor/src/test/scala/akka/transactor/FickleFriendsSpec.scala b/akka-transactor/src/test/scala/akka/transactor/FickleFriendsSpec.scala index 6f4e46de6a..a8e4e1ad0e 100644 --- a/akka-transactor/src/test/scala/akka/transactor/FickleFriendsSpec.scala +++ b/akka-transactor/src/test/scala/akka/transactor/FickleFriendsSpec.scala @@ -15,6 +15,7 @@ import akka.testkit.TestEvent.Mute import scala.concurrent.stm._ import scala.util.Random.{ nextInt ⇒ random } import java.util.concurrent.CountDownLatch +import akka.patterns.ask object FickleFriends { case class FriendlyIncrement(friends: Seq[ActorRef], timeout: Timeout, latch: CountDownLatch) diff --git a/akka-transactor/src/test/scala/akka/transactor/TransactorSpec.scala b/akka-transactor/src/test/scala/akka/transactor/TransactorSpec.scala index db273cdac7..2d5797c97a 100644 --- a/akka-transactor/src/test/scala/akka/transactor/TransactorSpec.scala +++ b/akka-transactor/src/test/scala/akka/transactor/TransactorSpec.scala @@ -10,6 +10,7 @@ import akka.util.duration._ import akka.util.Timeout import akka.testkit._ import scala.concurrent.stm._ +import akka.patterns.ask object TransactorIncrement { case class Increment(friends: Seq[ActorRef], latch: TestLatch) From 2f7b594e2849ed6f7f3d69cf1256ff0408a534c8 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Mon, 2 Jan 2012 22:04:37 -0700 Subject: [PATCH 010/152] IOManager is now an Extension --- akka-actor/src/main/scala/akka/actor/IO.scala | 133 ++++++++---------- 1 file changed, 61 insertions(+), 72 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/IO.scala b/akka-actor/src/main/scala/akka/actor/IO.scala index bf4a8fc84b..9e702238b0 100644 --- a/akka-actor/src/main/scala/akka/actor/IO.scala +++ b/akka-actor/src/main/scala/akka/actor/IO.scala @@ -76,7 +76,7 @@ object IO { case class Write(handle: WriteHandle, bytes: ByteString) extends IOMessage def listen(address: InetSocketAddress)(implicit system: ActorSystem, owner: ActorRef): ServerHandle = { - val ioManager = IOManager.start()(system) + val ioManager = IOManager(system).actor val server = ServerHandle(owner, ioManager) ioManager ! Listen(server, address) server @@ -92,7 +92,7 @@ object IO { listen(new InetSocketAddress(host, port))(system, owner) def connect(address: InetSocketAddress)(implicit system: ActorSystem, owner: ActorRef): SocketHandle = { - val ioManager = IOManager.start()(system) + val ioManager = IOManager(system).actor val socket = SocketHandle(owner, ioManager) ioManager ! Connect(socket, address) socket @@ -408,75 +408,17 @@ object IO { } -object IOManager { - def start()(implicit system: ActorSystem): ActorRef = { - // TODO: Replace with better "get or create" if/when available - val ref = system.actorFor(system / "io-manager") - if (!ref.isInstanceOf[EmptyLocalActorRef]) ref else try { - system.actorOf(Props[IOManager], "io-manager") - } catch { - case _: InvalidActorNameException ⇒ ref - } - } - def stop()(implicit system: ActorSystem): Unit = { - // TODO: send shutdown message to IOManager - } +final class IOManager(system: ActorSystem) extends Extension { + val actor = system.actorOf(Props[IOManagerActor], "io-manager") } -final class WriteBuffer(bufferSize: Int) { - private val _queue = new java.util.ArrayDeque[ByteString] - private val _buffer = ByteBuffer.allocate(bufferSize) - private var _length = 0 - - private def fillBuffer(): Boolean = { - while (!_queue.isEmpty && _buffer.hasRemaining) { - val next = _queue.pollFirst - val rest = next.drop(next.copyToBuffer(_buffer)) - if (rest.nonEmpty) _queue.offerFirst(rest) - } - !_buffer.hasRemaining - } - - def enqueue(elem: ByteString): this.type = { - _length += elem.length - val rest = elem.drop(elem.copyToBuffer(_buffer)) - if (rest.nonEmpty) _queue.offerLast(rest) - this - } - - def length = _length - - def isEmpty = _length == 0 - - def write(channel: WritableByteChannel with SelectableChannel): Int = { - @tailrec - def run(total: Int): Int = { - if (this.isEmpty) total - else { - val written = try { - _buffer.flip() - channel write _buffer - } finally { - // don't leave buffer in wrong state - _buffer.compact() - fillBuffer() - } - _length -= written - if (_buffer.position > 0) { - total + written - } else { - run(total + written) - } - } - } - - run(0) - } - +object IOManager extends ExtensionId[IOManager] with ExtensionIdProvider { + override def lookup = this + override def createExtension(system: ActorSystemImpl) = new IOManager(system) } // TODO: Support a pool of workers -final class IOManager extends Actor { +final class IOManagerActor extends Actor { import SelectionKey.{ OP_READ, OP_WRITE, OP_ACCEPT, OP_CONNECT } val bufferSize = 8192 // TODO: make buffer size configurable @@ -500,10 +442,6 @@ final class IOManager extends Actor { val selectAt = 100 // TODO: determine best value, perhaps based on throughput? Other triggers (like write queue size)? - //val selectEveryNanos = 1000000 // nanos - - //var lastSelectNanos = System.nanoTime - var running = false var selectSent = false @@ -521,7 +459,7 @@ final class IOManager extends Actor { } } lastSelect += 1 - if (lastSelect >= selectAt /* || (lastSelectNanos + selectEveryNanos) < System.nanoTime */ ) select() + if (lastSelect >= selectAt) select() } def select() { @@ -540,7 +478,6 @@ final class IOManager extends Actor { } else { running = false } - //lastSelectNanos = System.nanoTime lastSelect = 0 } @@ -721,3 +658,55 @@ final class IOManager extends Actor { } } + +final class WriteBuffer(bufferSize: Int) { + private val _queue = new java.util.ArrayDeque[ByteString] + private val _buffer = ByteBuffer.allocate(bufferSize) + private var _length = 0 + + private def fillBuffer(): Boolean = { + while (!_queue.isEmpty && _buffer.hasRemaining) { + val next = _queue.pollFirst + val rest = next.drop(next.copyToBuffer(_buffer)) + if (rest.nonEmpty) _queue.offerFirst(rest) + } + !_buffer.hasRemaining + } + + def enqueue(elem: ByteString): this.type = { + _length += elem.length + val rest = elem.drop(elem.copyToBuffer(_buffer)) + if (rest.nonEmpty) _queue.offerLast(rest) + this + } + + def length = _length + + def isEmpty = _length == 0 + + def write(channel: WritableByteChannel with SelectableChannel): Int = { + @tailrec + def run(total: Int): Int = { + if (this.isEmpty) total + else { + val written = try { + _buffer.flip() + channel write _buffer + } finally { + // don't leave buffer in wrong state + _buffer.compact() + fillBuffer() + } + _length -= written + if (_buffer.position > 0) { + total + written + } else { + run(total + written) + } + } + } + + run(0) + } + +} From 8894772282f4a7cefa89955ac6e91e49db2b662e Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Mon, 2 Jan 2012 22:10:00 -0700 Subject: [PATCH 011/152] clean up IO tests based on feedback --- .../src/test/scala/akka/actor/IOActor.scala | 16 +++++----------- 1 file changed, 5 insertions(+), 11 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala index 5c2c162f25..02e91eb02a 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala @@ -14,7 +14,7 @@ object IOActorSpec { class SimpleEchoServer(host: String, port: Int, started: TestLatch) extends Actor { - implicit val system = context.system + import context.system IO listen (host, port) @@ -26,13 +26,7 @@ object IOActorSpec { case IO.NewClient(server) ⇒ val socket = server.accept() - state(socket) flatMap { _ ⇒ - IO repeat { - IO.takeAny map { bytes ⇒ - socket write bytes - } - } - } + state(socket) flatMap (_ ⇒ IO repeat (IO.takeAny map socket.write)) case IO.Read(socket, bytes) ⇒ state(socket)(IO Chunk bytes) @@ -46,7 +40,7 @@ object IOActorSpec { class SimpleEchoClient(host: String, port: Int) extends Actor { - implicit val system = context.system + import context.system val socket = IO connect (host, port) @@ -91,7 +85,7 @@ object IOActorSpec { // Basic Redis-style protocol class KVStore(host: String, port: Int, started: TestLatch) extends Actor { - implicit val system = context.system + import context.system val state = IO.IterateeRef.Map.sync[IO.Handle]() @@ -157,7 +151,7 @@ object IOActorSpec { class KVClient(host: String, port: Int) extends Actor { - implicit val system = context.system + import context.system val socket = IO connect (host, port) From d2b8d915697ec550578fa77d1cccee8ed6181bc7 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Mon, 2 Jan 2012 22:23:28 -0700 Subject: [PATCH 012/152] Force use of IOManager factory methods to create default IOManager. If additional IOManagers are required in an ActorSystem the actors must be started manually --- akka-actor/src/main/scala/akka/actor/IO.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/actor/IO.scala b/akka-actor/src/main/scala/akka/actor/IO.scala index 9e702238b0..fbe560028c 100644 --- a/akka-actor/src/main/scala/akka/actor/IO.scala +++ b/akka-actor/src/main/scala/akka/actor/IO.scala @@ -408,7 +408,7 @@ object IO { } -final class IOManager(system: ActorSystem) extends Extension { +final class IOManager private (system: ActorSystem) extends Extension { val actor = system.actorOf(Props[IOManagerActor], "io-manager") } From a5611e88b2c628aaae70a380264a8bee5711b1ac Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Tue, 3 Jan 2012 13:18:31 -0700 Subject: [PATCH 013/152] move listen and connect into IOManager class --- .../src/test/scala/akka/actor/IOActor.scala | 14 ++--- akka-actor/src/main/scala/akka/actor/IO.scala | 51 +++++++------------ 2 files changed, 23 insertions(+), 42 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala index 02e91eb02a..434470fbd0 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala @@ -14,9 +14,7 @@ object IOActorSpec { class SimpleEchoServer(host: String, port: Int, started: TestLatch) extends Actor { - import context.system - - IO listen (host, port) + IOManager(context.system) listen (host, port) started.open @@ -40,9 +38,7 @@ object IOActorSpec { class SimpleEchoClient(host: String, port: Int) extends Actor { - import context.system - - val socket = IO connect (host, port) + val socket = IOManager(context.system) connect (host, port) val state = IO.IterateeRef.sync() @@ -91,7 +87,7 @@ object IOActorSpec { var kvs: Map[String, String] = Map.empty - IO listen (host, port) + IOManager(context.system) listen (host, port) started.open @@ -151,9 +147,7 @@ object IOActorSpec { class KVClient(host: String, port: Int) extends Actor { - import context.system - - val socket = IO connect (host, port) + val socket = IOManager(context.system) connect (host, port) val state = IO.IterateeRef.sync() diff --git a/akka-actor/src/main/scala/akka/actor/IO.scala b/akka-actor/src/main/scala/akka/actor/IO.scala index fbe560028c..d2d509f4aa 100644 --- a/akka-actor/src/main/scala/akka/actor/IO.scala +++ b/akka-actor/src/main/scala/akka/actor/IO.scala @@ -75,38 +75,6 @@ object IO { case class Read(handle: ReadHandle, bytes: ByteString) extends IOMessage case class Write(handle: WriteHandle, bytes: ByteString) extends IOMessage - def listen(address: InetSocketAddress)(implicit system: ActorSystem, owner: ActorRef): ServerHandle = { - val ioManager = IOManager(system).actor - val server = ServerHandle(owner, ioManager) - ioManager ! Listen(server, address) - server - } - - def listen(host: String, port: Int)(implicit system: ActorSystem, owner: ActorRef): ServerHandle = - listen(new InetSocketAddress(host, port))(system, owner) - - def listen(address: InetSocketAddress, owner: ActorRef)(implicit system: ActorSystem): ServerHandle = - listen(address)(system, owner) - - def listen(host: String, port: Int, owner: ActorRef)(implicit system: ActorSystem): ServerHandle = - listen(new InetSocketAddress(host, port))(system, owner) - - def connect(address: InetSocketAddress)(implicit system: ActorSystem, owner: ActorRef): SocketHandle = { - val ioManager = IOManager(system).actor - val socket = SocketHandle(owner, ioManager) - ioManager ! Connect(socket, address) - socket - } - - def connect(host: String, port: Int)(implicit system: ActorSystem, owner: ActorRef): SocketHandle = - connect(new InetSocketAddress(host, port))(system, owner) - - def connect(address: InetSocketAddress, owner: ActorRef)(implicit system: ActorSystem): SocketHandle = - connect(address)(system, owner) - - def connect(host: String, port: Int, owner: ActorRef)(implicit system: ActorSystem): SocketHandle = - connect(new InetSocketAddress(host, port))(system, owner) - sealed trait Input { def ++(that: Input): Input } @@ -410,6 +378,25 @@ object IO { final class IOManager private (system: ActorSystem) extends Extension { val actor = system.actorOf(Props[IOManagerActor], "io-manager") + + def listen(address: InetSocketAddress)(implicit owner: ActorRef): IO.ServerHandle = { + val server = IO.ServerHandle(owner, actor) + actor ! IO.Listen(server, address) + server + } + + def listen(host: String, port: Int)(implicit owner: ActorRef): IO.ServerHandle = + listen(new InetSocketAddress(host, port))(owner) + + def connect(address: InetSocketAddress)(implicit owner: ActorRef): IO.SocketHandle = { + val socket = IO.SocketHandle(owner, actor) + actor ! IO.Connect(socket, address) + socket + } + + def connect(host: String, port: Int)(implicit owner: ActorRef): IO.SocketHandle = + connect(new InetSocketAddress(host, port))(owner) + } object IOManager extends ExtensionId[IOManager] with ExtensionIdProvider { From 1fc35a04adee8782c44045db3ce2155f1e1f6837 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Tue, 3 Jan 2012 15:28:47 -0700 Subject: [PATCH 014/152] IO docs, in progress --- .../scala/code/akka/docs/io/HTTPServer.scala | 216 ++++++++++++++++++ akka-docs/scala/index.rst | 1 + akka-docs/scala/io.rst | 74 ++++++ 3 files changed, 291 insertions(+) create mode 100644 akka-docs/scala/code/akka/docs/io/HTTPServer.scala create mode 100644 akka-docs/scala/io.rst diff --git a/akka-docs/scala/code/akka/docs/io/HTTPServer.scala b/akka-docs/scala/code/akka/docs/io/HTTPServer.scala new file mode 100644 index 0000000000..b47b7ace44 --- /dev/null +++ b/akka-docs/scala/code/akka/docs/io/HTTPServer.scala @@ -0,0 +1,216 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka.docs.io + +import akka.actor._ +import akka.util.{ ByteString, ByteStringBuilder } +import java.net.InetSocketAddress + +class HttpServer(port: Int) extends Actor { + + val state = IO.IterateeRef.Map.async[IO.Handle]()(context.dispatcher) + + override def preStart { + IOManager(context.system) listen new InetSocketAddress(port) + } + + def receive = { + + case IO.NewClient(server) ⇒ + val socket = server.accept() + state(socket) flatMap (_ ⇒ HttpServer.processRequest(socket)) + + case IO.Read(socket, bytes) ⇒ + state(socket)(IO Chunk bytes) + + case IO.Closed(socket, cause) ⇒ + state(socket)(IO EOF None) + state -= socket + + } + +} + +object HttpServer { + import HttpIteratees._ + + def processRequest(socket: IO.SocketHandle): IO.Iteratee[Unit] = + IO repeat { + for { + request ← readRequest + } yield { + val rsp = request match { + case Request("GET", "ping" :: Nil, _, _, headers, _) ⇒ + OKResponse(ByteString("

pong

"), + request.headers.exists { case Header(n, v) ⇒ n.toLowerCase == "connection" && v.toLowerCase == "keep-alive" }) + case req ⇒ + OKResponse(ByteString("

" + req.toString + "

"), + request.headers.exists { case Header(n, v) ⇒ n.toLowerCase == "connection" && v.toLowerCase == "keep-alive" }) + } + socket write OKResponse.bytes(rsp).compact + if (!rsp.keepAlive) socket.close() + } + } + +} + +//#request-class +case class Request(meth: String, path: List[String], query: Option[String], httpver: String, headers: List[Header], body: Option[ByteString]) +case class Header(name: String, value: String) +//#request-class + +//#constants +object HttpConstants { + val SP = ByteString(" ") + val HT = ByteString("\t") + val CRLF = ByteString("\r\n") + val COLON = ByteString(":") + val PERCENT = ByteString("%") + val PATH = ByteString("/") + val QUERY = ByteString("?") +} +//#constants + +//#read-request +object HttpIteratees { + import HttpConstants._ + + def readRequest = + for { + requestLine ← readRequestLine + (meth, (path, query), httpver) = requestLine + headers ← readHeaders + body ← readBody(headers) + } yield Request(meth, path, query, httpver, headers, body) + //#read-request + + //#read-request-line + def ascii(bytes: ByteString): String = bytes.decodeString("US-ASCII").trim + + def readRequestLine = + for { + meth ← IO takeUntil SP + uri ← readRequestURI + _ ← IO takeUntil SP // ignore the rest + httpver ← IO takeUntil CRLF + } yield (ascii(meth), uri, ascii(httpver)) + //#read-request-line + + //#read-request-uri + def readRequestURI = IO peek 1 flatMap { + case PATH ⇒ + for { + path ← readPath + query ← readQuery + } yield (path, query) + case _ ⇒ sys.error("Not Implemented") + } + //#read-request-uri + + //#read-path + def readPath = { + def step(segments: List[String]): IO.Iteratee[List[String]] = IO peek 1 flatMap { + case PATH ⇒ IO drop 1 flatMap (_ ⇒ readUriPart(pathchar) flatMap (segment ⇒ step(segment :: segments))) + case _ ⇒ segments match { + case "" :: rest ⇒ IO Done rest.reverse + case _ ⇒ IO Done segments.reverse + } + } + step(Nil) + } + //#read-path + + //#read-query + def readQuery: IO.Iteratee[Option[String]] = IO peek 1 flatMap { + case QUERY ⇒ IO drop 1 flatMap (_ ⇒ readUriPart(querychar) map (Some(_))) + case _ ⇒ IO Done None + } + //#read-query + + //#read-uri-part + val alpha = Set.empty ++ ('a' to 'z') ++ ('A' to 'Z') map (_.toByte) + val digit = Set.empty ++ ('0' to '9') map (_.toByte) + val hexdigit = digit ++ (Set.empty ++ ('a' to 'f') ++ ('A' to 'F') map (_.toByte)) + val subdelim = Set('!', '$', '&', '\'', '(', ')', '*', '+', ',', ';', '=') map (_.toByte) + val pathchar = alpha ++ digit ++ subdelim ++ (Set(':', '@') map (_.toByte)) + val querychar = pathchar ++ (Set('/', '?') map (_.toByte)) + + def readUriPart(allowed: Set[Byte]): IO.Iteratee[String] = for { + str ← IO takeWhile allowed map ascii + pchar ← IO peek 1 map (_ == PERCENT) + all ← if (pchar) readPChar flatMap (ch ⇒ readUriPart(allowed) map (str + ch + _)) else IO Done str + } yield all + + def readPChar = IO take 3 map { + case Seq('%', rest @ _*) if rest forall hexdigit ⇒ + java.lang.Integer.parseInt(rest map (_.toChar) mkString, 16).toChar + } + //#read-uri-part + + //#read-headers + def readHeaders = { + def step(found: List[Header]): IO.Iteratee[List[Header]] = { + IO peek 2 flatMap { + case CRLF ⇒ IO takeUntil CRLF flatMap (_ ⇒ IO Done found) + case _ ⇒ readHeader flatMap (header ⇒ step(header :: found)) + } + } + step(Nil) + } + + def readHeader = + for { + name ← IO takeUntil COLON + value ← IO takeUntil CRLF flatMap readMultiLineValue + } yield Header(ascii(name), ascii(value)) + + def readMultiLineValue(initial: ByteString): IO.Iteratee[ByteString] = IO peek 1 flatMap { + case SP ⇒ IO takeUntil CRLF flatMap (bytes ⇒ readMultiLineValue(initial ++ bytes)) + case _ ⇒ IO Done initial + } + //#read-headers + + //#read-body + def readBody(headers: List[Header]) = + if (headers.exists(header ⇒ header.name == "Content-Length" || header.name == "Transfer-Encoding")) + IO.takeAll map (Some(_)) + else + IO Done None + //#read-body +} + +//#ok-response +object OKResponse { + import HttpConstants.CRLF + + val okStatus = ByteString("HTTP/1.1 200 OK") + val contentType = ByteString("Content-Type: text/html; charset=utf-8") + val cacheControl = ByteString("Cache-Control: no-cache") + val date = ByteString("Date: ") + val server = ByteString("Server: Akka") + val contentLength = ByteString("Content-Length: ") + val connection = ByteString("Connection: ") + val keepAlive = ByteString("Keep-Alive") + val close = ByteString("Close") + + def bytes(rsp: OKResponse) = { + new ByteStringBuilder ++= + okStatus ++= CRLF ++= + contentType ++= CRLF ++= + cacheControl ++= CRLF ++= + date ++= ByteString(new java.util.Date().toString) ++= CRLF ++= + server ++= CRLF ++= + contentLength ++= ByteString(rsp.body.length.toString) ++= CRLF ++= + connection ++= (if (rsp.keepAlive) keepAlive else close) ++= CRLF ++= CRLF ++= rsp.body result + } + +} +case class OKResponse(body: ByteString, keepAlive: Boolean) +//#ok-response + +object Main extends App { + val port = Option(System.getenv("PORT")) map (_.toInt) getOrElse 8080 + val system = ActorSystem() + val server = system.actorOf(Props(new HttpServer(port))) +} diff --git a/akka-docs/scala/index.rst b/akka-docs/scala/index.rst index 803dd26799..268fc06981 100644 --- a/akka-docs/scala/index.rst +++ b/akka-docs/scala/index.rst @@ -22,5 +22,6 @@ Scala API stm agents transactors + io testing extending-akka diff --git a/akka-docs/scala/io.rst b/akka-docs/scala/io.rst new file mode 100644 index 0000000000..2379902716 --- /dev/null +++ b/akka-docs/scala/io.rst @@ -0,0 +1,74 @@ +.. _io-scala: + +IO (Scala) +========== + +.. sidebar:: Contents + + .. contents:: :local: + +Introduction +------------ + +This documentation is in progress. More to come. + +Examples +-------- + +Http Server +^^^^^^^^^^^ + +Some commonly used constants: + +.. includecode:: code/akka/docs/io/HTTPServer.scala + :include: constants + +And case classes to hold the resulting request: + +.. includecode:: code/akka/docs/io/HTTPServer.scala + :include: request-class + +Now for our first ``Iteratee``. There are 3 main sections of a HTTP request: the request line, the headers, and an optional body. The main request ``Iteratee`` handles each section separately: + +.. includecode:: code/akka/docs/io/HTTPServer.scala + :include: read-request + +In the above code ``readRequest`` takes the results of 3 different ``Iteratees`` (``readRequestLine``, ``readHeaders``, ``readBody``) and combines them into a single ``Request`` object. ``readRequestLine`` actually returns a tuple, so we extract it's individual components. ``readBody`` depends on values contained within the header section, so we must pass those to the method. + +The request line has 3 parts to it: the HTTP method, the requested URI, and the HTTP version. The parts are separated by a single space, and the entire request line ends with a ``CRLF``. + +.. includecode:: code/akka/docs/io/HTTPServer.scala + :include: read-request-line + +Reading the request method is simple as it is a single string ending in a space. The simple ``Iteratee`` that performs this is ``IO.takeUntil(delimiter: ByteString): Iteratee[ByteString]``. It keeps consuming input until the specified delimiter is found. Reading the HTTP version is also a simple string that ends with a ``CRLF``. + +The ``ascii`` method is a helper that takes a ``ByteString`` and parses it as a ``US-ASCII`` ``String``. + +Reading the request URI is a bit more complicated because we want to parse the individual components of the URI instead of just returning a simple string: + +.. includecode:: code/akka/docs/io/HTTPServer.scala + :include: read-request-uri + +For this example we are only interested in handling absolute paths. To detect if we the URI is an absolute path we use ``IO.peek(length: Int): Iteratee[ByteString]``, which returns a ``ByteString`` of the request length but doesn't actually consume the input. We peek at the next bit of input and see if it matches our ``PATH`` constant (defined above as ``ByteString("/")``). If it doesn't match we throw an error, but for a more robust solution we would want to handle other valid URIs. + +Reading the URI path will be our most complex ``Iteratee``. It involves a recursive method that reads in each path segment of the URI: + +.. includecode:: code/akka/docs/io/HTTPServer.scala + :include: read-path + +The ``step`` method is a recursive method that takes a ``List`` of the accumulated path segments. It first checks if the remaining input starts with the ``PATH`` constant, and if it does, it drops that input, and returns the ``readUriPart`` ``Iteratee`` which has it's result added to the path segment accumulator and the ``step`` method is run again. + +If after reading in a path segment the next input does not start with a path, we reverse the accumulated segments and return it (dropping the last segment if it is blank). + +.. includecode:: code/akka/docs/io/HTTPServer.scala + :include: read-query + +.. includecode:: code/akka/docs/io/HTTPServer.scala + :include: read-uri-part + +.. includecode:: code/akka/docs/io/HTTPServer.scala + :include: read-headers + +.. includecode:: code/akka/docs/io/HTTPServer.scala + :include: read-body + From 64e0b9eb526550105db01133b7562858550cd1aa Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Sat, 14 Jan 2012 10:31:06 -0700 Subject: [PATCH 015/152] Initial attempt to allow handling errors in iteratees --- .../src/test/scala/akka/actor/IOActor.scala | 32 ++++----- akka-actor/src/main/scala/akka/actor/IO.scala | 69 ++++++++++--------- 2 files changed, 53 insertions(+), 48 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala index 434470fbd0..a8408eb92f 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala @@ -12,12 +12,10 @@ import akka.dispatch.{ Await, Future } object IOActorSpec { - class SimpleEchoServer(host: String, port: Int, started: TestLatch) extends Actor { + class SimpleEchoServer(host: String, port: Int) extends Actor { IOManager(context.system) listen (host, port) - started.open - val state = IO.IterateeRef.Map.sync[IO.Handle]() def receive = { @@ -38,7 +36,7 @@ object IOActorSpec { class SimpleEchoClient(host: String, port: Int) extends Actor { - val socket = IOManager(context.system) connect (host, port) + var socket = IOManager(context.system) connect (host, port) val state = IO.IterateeRef.sync() @@ -47,17 +45,21 @@ object IOActorSpec { case bytes: ByteString ⇒ val source = sender socket write bytes - for { - _ ← state - bytes ← IO take bytes.length - } yield source ! bytes + state flatMap { _ ⇒ + IO take bytes.length map (source ! _) recover { + case e: java.net.ConnectException ⇒ + self.tell(bytes, source) + } + } - case IO.Read(socket, bytes) ⇒ + case IO.Read(_, bytes) ⇒ state(IO Chunk bytes) - case IO.Connected(socket) ⇒ + case IO.Connected(_) ⇒ - case IO.Closed(socket, cause) ⇒ + case IO.Closed(_, cause) ⇒ + state(IO EOF cause) + socket = IOManager(context.system) connect (host, port) } } @@ -208,23 +210,19 @@ class IOActorSpec extends AkkaSpec with DefaultTimeout { "an IO Actor" must { "run echo server" in { - val started = TestLatch(1) - val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8064, started))) - Await.ready(started, TestLatch.DefaultTimeout) val client = system.actorOf(Props(new SimpleEchoClient("localhost", 8064))) val f1 = client ? ByteString("Hello World!1") val f2 = client ? ByteString("Hello World!2") val f3 = client ? ByteString("Hello World!3") + val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8064))) Await.result(f1, TestLatch.DefaultTimeout) must equal(ByteString("Hello World!1")) Await.result(f2, TestLatch.DefaultTimeout) must equal(ByteString("Hello World!2")) Await.result(f3, TestLatch.DefaultTimeout) must equal(ByteString("Hello World!3")) } "run echo server under high load" in { - val started = TestLatch(1) - val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8065, started))) - Await.ready(started, TestLatch.DefaultTimeout) val client = system.actorOf(Props(new SimpleEchoClient("localhost", 8065))) + val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8065))) val list = List.range(0, 1000) val f = Future.traverse(list)(i ⇒ client ? ByteString(i.toString)) assert(Await.result(f, TestLatch.DefaultTimeout).size === 1000) diff --git a/akka-actor/src/main/scala/akka/actor/IO.scala b/akka-actor/src/main/scala/akka/actor/IO.scala index d2d509f4aa..5942f285de 100644 --- a/akka-actor/src/main/scala/akka/actor/IO.scala +++ b/akka-actor/src/main/scala/akka/actor/IO.scala @@ -91,7 +91,7 @@ object IO { } case class EOF(cause: Option[Exception]) extends Input { - def ++(that: Input) = this + def ++(that: Input) = that } object Iteratee { @@ -111,28 +111,32 @@ object IO { * and the unused Input. */ final def apply(input: Input): (Iteratee[A], Input) = this match { - case Cont(f) ⇒ f(input) - case iter ⇒ (iter, input) + case Cont(f, None) ⇒ f(input) + case iter ⇒ (iter, input) } final def get: A = this(EOF(None))._1 match { - case Done(value) ⇒ value - case Cont(_) ⇒ sys.error("Divergent Iteratee") - case Failure(e) ⇒ throw e + case Done(value) ⇒ value + case Cont(_, None) ⇒ sys.error("Divergent Iteratee") + case Cont(_, Some(err)) ⇒ throw err } final def flatMap[B](f: A ⇒ Iteratee[B]): Iteratee[B] = this match { - case Done(value) ⇒ f(value) - case Cont(k: Chain[_]) ⇒ Cont(k :+ f) - case Cont(k) ⇒ Cont(Chain(k, f)) - case failure: Failure ⇒ failure + case Done(value) ⇒ f(value) + case Cont(k: Chain[_], err) ⇒ Cont(k :+ f, err) + case Cont(k, err) ⇒ Cont(Chain(k, f), err) } final def map[B](f: A ⇒ B): Iteratee[B] = this match { - case Done(value) ⇒ Done(f(value)) - case Cont(k: Chain[_]) ⇒ Cont(k :+ ((a: A) ⇒ Done(f(a)))) - case Cont(k) ⇒ Cont(Chain(k, (a: A) ⇒ Done(f(a)))) - case failure: Failure ⇒ failure + case Done(value) ⇒ Done(f(value)) + case Cont(k: Chain[_], err) ⇒ Cont(k :+ ((a: A) ⇒ Done(f(a))), err) + case Cont(k, err) ⇒ Cont(Chain(k, (a: A) ⇒ Done(f(a))), err) + } + + def recover[B >: A](pf: PartialFunction[Exception, B]): Iteratee[B] = this match { + case done @ Done(_) ⇒ done + case Cont(_, Some(err)) if pf isDefinedAt err ⇒ Done(pf(err)) + case Cont(k, err) ⇒ Cont((more ⇒ k(more) match { case (iter, rest) ⇒ (iter recover pf, rest) }), err) } } @@ -147,13 +151,7 @@ object IO { /** * An Iteratee that still requires more input to calculate it's result. */ - final case class Cont[+A](f: Input ⇒ (Iteratee[A], Input)) extends Iteratee[A] - - /** - * An Iteratee representing a failure to calcualte a result. - * FIXME: move into 'Cont' as in Oleg's implementation - */ - final case class Failure(exception: Throwable) extends Iteratee[Nothing] + final case class Cont[+A](f: Input ⇒ (Iteratee[A], Input), error: Option[Exception] = None) extends Iteratee[A] object IterateeRef { def sync[A](initial: Iteratee[A]): IterateeRefSync[A] = new IterateeRefSync(initial) @@ -212,6 +210,8 @@ object IO { def future: Future[(Iteratee[A], Input)] = _value } + final def throwErr(err: Exception): Iteratee[Nothing] = Cont(input ⇒ (throwErr(err), input), Some(err)) + /** * An Iteratee that returns the ByteString prefix up until the supplied delimiter. * The delimiter is dropped by default, but it can be returned with the result by @@ -228,7 +228,8 @@ object IO { } else { (Cont(step(bytes)), Chunk.empty) } - case eof ⇒ (Cont(step(taken)), eof) + case eof @ EOF(None) ⇒ (Done(taken), eof) + case eof @ EOF(cause) ⇒ (Cont(step(taken), cause), eof) } Cont(step(ByteString.empty)) @@ -242,7 +243,8 @@ object IO { (Cont(step(taken ++ found)), Chunk.empty) else (Done(taken ++ found), Chunk(rest)) - case eof ⇒ (Done(taken), eof) + case eof @ EOF(None) ⇒ (Done(taken), eof) + case eof @ EOF(cause) ⇒ (Cont(step(taken), cause), eof) } Cont(step(ByteString.empty)) @@ -259,7 +261,8 @@ object IO { (Done(bytes.take(length)), Chunk(bytes.drop(length))) else (Cont(step(bytes)), Chunk.empty) - case eof ⇒ (Cont(step(taken)), eof) + case eof @ EOF(None) ⇒ (Done(taken), eof) + case eof @ EOF(cause) ⇒ (Cont(step(taken), cause), eof) } Cont(step(ByteString.empty)) @@ -275,7 +278,8 @@ object IO { (Cont(step(left - more.length)), Chunk.empty) else (Done(), Chunk(more drop left)) - case eof ⇒ (Done(), eof) + case eof @ EOF(None) ⇒ (Done(), eof) + case eof @ EOF(cause) ⇒ (Cont(step(left), cause), eof) } Cont(step(length)) @@ -289,7 +293,8 @@ object IO { case Chunk(more) ⇒ val bytes = taken ++ more (Cont(step(bytes)), Chunk.empty) - case eof ⇒ (Done(taken), eof) + case eof @ EOF(None) ⇒ (Done(taken), eof) + case eof @ EOF(cause) ⇒ (Cont(step(taken), cause), eof) } Cont(step(ByteString.empty)) @@ -301,7 +306,8 @@ object IO { val takeAny: Iteratee[ByteString] = Cont { case Chunk(bytes) if bytes.nonEmpty ⇒ (Done(bytes), Chunk.empty) case Chunk(bytes) ⇒ (takeAny, Chunk.empty) - case eof ⇒ (Done(ByteString.empty), eof) + case eof @ EOF(None) ⇒ (Done(ByteString.empty), eof) + case eof @ EOF(cause) ⇒ (Cont(more ⇒ (Done(ByteString.empty), more), cause), eof) } def takeList[A](length: Int)(iter: Iteratee[A]): Iteratee[List[A]] = { @@ -320,7 +326,8 @@ object IO { (Done(bytes.take(length)), Chunk(bytes)) else (Cont(step(bytes)), Chunk.empty) - case eof ⇒ (Cont(step(taken)), eof) + case eof @ EOF(None) ⇒ (Done(taken), eof) + case eof @ EOF(cause) ⇒ (Cont(step(taken), cause), eof) } Cont(step(ByteString.empty)) @@ -362,10 +369,10 @@ object IO { case (Done(value), rest) ⇒ queueOut.head(value) match { //case Cont(Chain(f, q)) ⇒ run(f(rest), q ++ tail) <- can cause big slowdown, need to test if needed - case Cont(f) ⇒ run(f(rest), queueOut.tail, queueIn) - case iter ⇒ run((iter, rest), queueOut.tail, queueIn) + case Cont(f, None) ⇒ run(f(rest), queueOut.tail, queueIn) + case iter ⇒ run((iter, rest), queueOut.tail, queueIn) } - case (Cont(f), rest) ⇒ + case (Cont(f, None), rest) ⇒ (Cont(new Chain(f, queueOut, queueIn)), rest) case _ ⇒ result } From c0a3441442bb26e57d71ecfdcd3d32d57507da41 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Sat, 14 Jan 2012 10:39:55 -0700 Subject: [PATCH 016/152] Echo client may send requests out of order on errors, okay for this example but will try to find way of maintaining order --- akka-actor-tests/src/test/scala/akka/actor/IOActor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala index a8408eb92f..f16e2b7fed 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala @@ -222,9 +222,9 @@ class IOActorSpec extends AkkaSpec with DefaultTimeout { "run echo server under high load" in { val client = system.actorOf(Props(new SimpleEchoClient("localhost", 8065))) - val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8065))) val list = List.range(0, 1000) val f = Future.traverse(list)(i ⇒ client ? ByteString(i.toString)) + val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8065))) assert(Await.result(f, TestLatch.DefaultTimeout).size === 1000) } From 6b8e8ea1e889413c9005e562600743f93f35e620 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Mon, 16 Jan 2012 23:38:36 -0700 Subject: [PATCH 017/152] add retry method in IOActorSpecs until I find a better home --- .../src/test/scala/akka/actor/IOActor.scala | 57 ++++++++++++++++--- 1 file changed, 48 insertions(+), 9 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala index f16e2b7fed..a7f06d9eb2 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala @@ -4,11 +4,11 @@ package akka.actor -import akka.util.ByteString -import akka.util.cps._ +import akka.util.{ ByteString, Duration, Timer } +import akka.util.duration._ import scala.util.continuations._ import akka.testkit._ -import akka.dispatch.{ Await, Future } +import akka.dispatch.{ Await, Future, Promise, ExecutionContext, MessageDispatcher } object IOActorSpec { @@ -47,8 +47,7 @@ object IOActorSpec { socket write bytes state flatMap { _ ⇒ IO take bytes.length map (source ! _) recover { - case e: java.net.ConnectException ⇒ - self.tell(bytes, source) + case e ⇒ source ! Status.Failure(e) } } @@ -208,12 +207,52 @@ object IOActorSpec { class IOActorSpec extends AkkaSpec with DefaultTimeout { import IOActorSpec._ + /** + * Retries the future until a result is returned or until one of the limits are hit. If no + * limits are provided the future will be retried indefinitely until a result is returned. + * + * @param count number of retries + * @param timeout duration to retry within + * @param delay duration to wait before retrying + * @param filter determines which exceptions should be retried + * @return a future containing the result or the last exception before a limit was hit. + */ + def retry[T](count: Option[Int] = None, timeout: Option[Duration] = None, delay: Option[Duration] = None, filter: Option[Throwable ⇒ Boolean] = None)(future: ⇒ Future[T])(implicit executor: ExecutionContext): Future[T] = { + + val promise = Promise[T]()(executor) + + val timer = timeout match { + case Some(duration) ⇒ Some(Timer(duration)) + case None ⇒ None + } + + def check(n: Int, e: Throwable): Boolean = + (count.isEmpty || (n < count.get)) && (timer.isEmpty || timer.get.isTicking) && (filter.isEmpty || filter.get(e)) + + def run(n: Int) { + future onComplete { + case Left(e) if check(n, e) ⇒ + if (delay.isDefined) { + executor match { + case m: MessageDispatcher ⇒ m.prerequisites.scheduler.scheduleOnce(delay.get)(run(n + 1)) + case _ ⇒ // Thread.sleep, ignore, or other? + } + } else run(n + 1) + case v ⇒ promise complete v + } + } + + run(0) + + promise + } + "an IO Actor" must { "run echo server" in { val client = system.actorOf(Props(new SimpleEchoClient("localhost", 8064))) - val f1 = client ? ByteString("Hello World!1") - val f2 = client ? ByteString("Hello World!2") - val f3 = client ? ByteString("Hello World!3") + val f1 = retry() { client ? ByteString("Hello World!1") } + val f2 = retry() { client ? ByteString("Hello World!2") } + val f3 = retry() { client ? ByteString("Hello World!3") } val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8064))) Await.result(f1, TestLatch.DefaultTimeout) must equal(ByteString("Hello World!1")) Await.result(f2, TestLatch.DefaultTimeout) must equal(ByteString("Hello World!2")) @@ -223,7 +262,7 @@ class IOActorSpec extends AkkaSpec with DefaultTimeout { "run echo server under high load" in { val client = system.actorOf(Props(new SimpleEchoClient("localhost", 8065))) val list = List.range(0, 1000) - val f = Future.traverse(list)(i ⇒ client ? ByteString(i.toString)) + val f = Future.traverse(list)(i ⇒ retry() { client ? ByteString(i.toString) }) val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8065))) assert(Await.result(f, TestLatch.DefaultTimeout).size === 1000) } From 9c762dec2062f7a651363a6cc90d8105126d8f35 Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 17 Jan 2012 17:04:20 +0100 Subject: [PATCH 018/152] =?UTF-8?q?polish=20=E2=80=9Cask=E2=80=9D=20patter?= =?UTF-8?q?n,=20see=20#1581?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - move package objects into their respective package.scala file in the right directories - make implicit conversion as well as explicit facility available under the same name akka.patterns.ask for easy import - revert the logic to produce the Promise for the PromiseActorRef within the ActorRefProvider; supporting wrapping of external Promises does not seem to justify doing needless extra allocations in case of failure - add scaladocs - factor out “def provider” into trait ActorRefWithProvider, as it didn’t feel right attaching this information “by exception” to MinimalActorRef --- .../scala/akka/actor/ForwardActorSpec.scala | 2 +- akka-actor/src/main/scala/akka/Patterns.scala | 71 ++++++++++ .../src/main/scala/akka/actor/ActorRef.scala | 19 ++- .../scala/akka/actor/ActorRefProvider.scala | 23 +--- .../src/main/scala/akka/actor/package.scala | 123 ------------------ .../scala/akka/patterns/AskableActorRef.scala | 84 ++++++++++++ .../main/scala/akka/patterns/package.scala | 70 ++++++++++ .../src/main/scala/akka/routing/Routing.scala | 14 +- .../code/akka/docs/actor/ActorDocSpec.scala | 7 +- .../code/akka/docs/future/FutureDocSpec.scala | 25 ++-- .../akka/remote/RemoteActorRefProvider.scala | 6 +- 11 files changed, 268 insertions(+), 176 deletions(-) create mode 100644 akka-actor/src/main/scala/akka/Patterns.scala create mode 100644 akka-actor/src/main/scala/akka/patterns/AskableActorRef.scala create mode 100644 akka-actor/src/main/scala/akka/patterns/package.scala diff --git a/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala index 10df9d2218..a019b4afb3 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala @@ -47,7 +47,7 @@ class ForwardActorSpec extends AkkaSpec { "forward actor reference when invoking forward on ask" in { val chain = createForwardingChain(system) - chain.ask(ExpectedMessage, 5000) onSuccess { case ExpectedMessage ⇒ testActor ! ExpectedMessage } + chain.ask(ExpectedMessage)(5000) onSuccess { case ExpectedMessage ⇒ testActor ! ExpectedMessage } expectMsg(5 seconds, ExpectedMessage) } } diff --git a/akka-actor/src/main/scala/akka/Patterns.scala b/akka-actor/src/main/scala/akka/Patterns.scala new file mode 100644 index 0000000000..7650c4077f --- /dev/null +++ b/akka-actor/src/main/scala/akka/Patterns.scala @@ -0,0 +1,71 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka + +object Patterns { + import akka.actor.ActorRef + import akka.dispatch.Future + import akka.patterns.{ ask ⇒ scalaAsk } + import akka.util.Timeout + + /** + * 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 + * given timeout has expired; this is independent from any timeout applied + * while awaiting a result for this future (i.e. in + * `Await.result(..., timeout)`). + * + * Warning: + * When using future callbacks, inside actors you need to carefully avoid closing over + * the containing actor’s object, i.e. do not call methods or access mutable state + * on the enclosing actor from within the callback. This would break the actor + * encapsulation and may introduce synchronization bugs and race conditions because + * the callback will be scheduled concurrently to the enclosing actor. Unfortunately + * there is not yet a way to detect these illegal accesses at compile time. + * + * Recommended usage: + * + * {{{ + * final Future f = Patterns.ask(worker, request, timeout); + * f.onSuccess(new Procedure() { + * public void apply(Object o) { + * nextActor.tell(new EnrichedResult(request, o)); + * } + * }); + * }}} + */ + def ask(actor: ActorRef, message: Any, timeout: Timeout): Future[AnyRef] = scalaAsk(actor, message)(timeout).asInstanceOf[Future[AnyRef]] + + /** + * 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 + * given timeout has expired; this is independent from any timeout applied + * while awaiting a result for this future (i.e. in + * `Await.result(..., timeout)`). + * + * Warning: + * When using future callbacks, inside actors you need to carefully avoid closing over + * the containing actor’s object, i.e. do not call methods or access mutable state + * on the enclosing actor from within the callback. This would break the actor + * encapsulation and may introduce synchronization bugs and race conditions because + * the callback will be scheduled concurrently to the enclosing actor. Unfortunately + * there is not yet a way to detect these illegal accesses at compile time. + * + * Recommended usage: + * + * {{{ + * final Future f = Patterns.ask(worker, request, timeout); + * f.onSuccess(new Procedure() { + * public void apply(Object o) { + * nextActor.tell(new EnrichedResult(request, o)); + * } + * }); + * }}} + */ + def ask(actor: ActorRef, message: Any, timeoutMillis: Long): Future[AnyRef] = scalaAsk(actor, message)(new Timeout(timeoutMillis)).asInstanceOf[Future[AnyRef]] +} \ No newline at end of file diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index b689aa5370..e7c03e8ae0 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -167,6 +167,13 @@ trait LocalRef extends ActorRefScope { final def isLocal = true } +/** + * Trait for matching on ActorRefs which have access to a provider; this is used in akka.patterns.ask. + */ +trait ActorRefWithProvider { this: InternalActorRef ⇒ + def provider: ActorRefProvider +} + /** * Internal trait for assembling all the functionality needed internally on * ActorRefs. NOTE THAT THIS IS NOT A STABLE EXTERNAL INTERFACE! @@ -180,7 +187,6 @@ private[akka] abstract class InternalActorRef extends ActorRef with ScalaActorRe def stop(): Unit def sendSystemMessage(message: SystemMessage): Unit def getParent: InternalActorRef - def provider: ActorRefProvider /** * Obtain ActorRef by possibly traversing the actor tree or looking it up at * some provider-specific location. This method shall return the end result, @@ -212,7 +218,7 @@ private[akka] class LocalActorRef private[akka] ( val systemService: Boolean = false, _receiveTimeout: Option[Duration] = None, _hotswap: Stack[PartialFunction[Any, Unit]] = Props.noHotSwap) - extends InternalActorRef with LocalRef { + extends InternalActorRef with LocalRef with ActorRefWithProvider { /* * actorCell.start() publishes actorCell & this to the dispatcher, which @@ -341,8 +347,7 @@ case class SerializedActorRef(path: String) { trait MinimalActorRef extends InternalActorRef with LocalRef { def getParent: InternalActorRef = Nobody - def provider: ActorRefProvider = - throw new UnsupportedOperationException("Not supported for [%s]".format(getClass.getName)) + def getChild(names: Iterator[String]): InternalActorRef = { val dropped = names.dropWhile(_.isEmpty) if (dropped.isEmpty) this @@ -466,10 +471,14 @@ class AskTimeoutException(message: String, cause: Throwable) extends TimeoutExce def this(message: String) = this(message, null: Throwable) } +/** + * Akka private optimized representation of the temporary actor spawned to + * receive the reply to an "ask" operation. + */ private[akka] final class PromiseActorRef( val path: ActorPath, override val getParent: InternalActorRef, - private final val result: Promise[Any], + val result: Promise[Any], val deathWatch: DeathWatch) extends MinimalActorRef { final val running = new AtomicBoolean(true) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 6af8c36198..4408f7562e 100755 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -104,7 +104,7 @@ trait ActorRefProvider { * Create AskActorRef and register it properly so it can be serialized/deserialized; * caller needs to send the message. */ - def ask(result: Promise[Any], within: Timeout): Option[ActorRef] + def ask(within: Timeout): Option[PromiseActorRef] /** * This Future is completed upon termination of this ActorRefProvider, which @@ -494,12 +494,13 @@ class LocalActorRefProvider( } } - def ask(result: Promise[Any], within: Timeout): Option[ActorRef] = { + def ask(within: Timeout): Option[PromiseActorRef] = { (if (within == null) settings.ActorTimeout else within) match { case t if t.duration.length <= 0 ⇒ None case t ⇒ val path = tempPath() val name = path.name + val result = Promise[Any]()(dispatcher) val a = new PromiseActorRef(path, tempContainer, result, deathWatch) tempContainer.addChild(name, a) val f = dispatcher.prerequisites.scheduler.scheduleOnce(t.duration) { result.failure(new AskTimeoutException("Timed out")) } @@ -509,24 +510,6 @@ class LocalActorRefProvider( } Some(a) - - // Alternative implementation: - // Create a full-blown actor to complete the promise. - // This would also work but not as efficient as PromiseActorRef. - //val b = actorOf(system, Props(new Actor { - // def receive = { - // case Status.Success(r) ⇒ result.success(r) - // case Status.Failure(f) ⇒ result.failure(f) - // case other ⇒ result.success(other) - // } - //}), systemGuardian, systemGuardian.path / "promise" / tempName(), false, None) - //val ff = system.scheduler.scheduleOnce(t.duration) { result.failure(new AskTimeoutException("Timed out")) } - //result onComplete { _ ⇒ - // b.stop() - // ff.cancel() - //} - // - //Some(b) } } } diff --git a/akka-actor/src/main/scala/akka/actor/package.scala b/akka-actor/src/main/scala/akka/actor/package.scala index acfbb19e5d..cfe5bc1b0d 100644 --- a/akka-actor/src/main/scala/akka/actor/package.scala +++ b/akka-actor/src/main/scala/akka/actor/package.scala @@ -39,126 +39,3 @@ package object actor { } } - -package object patterns { - - import akka.actor.{ ActorRef, InternalActorRef } - import akka.dispatch.Promise - import akka.util.Timeout - - implicit def ask(actorRef: ActorRef): AskableActorRef = new AskableActorRef()(actorRef) - - // Implicit for converting a Promise to an ActorRef. - // Symmetric to the future2actor conversion, which allows - // piping a Future result (read side) to an Actor's mailbox, this - // conversion allows using an Actor to complete a Promise (write side) - // - // Future.ask / actor ? message is now a trivial implementation that can - // also be done in user code (assuming actorRef, timeout and dispatcher implicits): - // - // Patterns.ask(actor, message) = { - // val promise = Promise[Any]() - // actor ! (message, promise) - // promise - // } - - @inline implicit def promise2actorRef(promise: Promise[Any])(implicit actorRef: ActorRef, timeout: Timeout): ActorRef = { - val provider = actorRef.asInstanceOf[InternalActorRef].provider - provider.ask(promise, timeout) match { - case Some(ref) ⇒ ref - case None ⇒ null - } - } - -} - -package patterns { - - import akka.actor.{ ActorRef, InternalActorRef } - import akka.dispatch.{ Future, Promise } - import akka.util.Timeout - - final class AskableActorRef(implicit val actorRef: ActorRef) { - - /** - * Akka Java API. - * - * Sends a message asynchronously returns a future holding the eventual reply message. - * The Future will be completed with an [[akka.actor.AskTimeoutException]] after the given - * timeout has expired. - * - * NOTE: - * Use this method with care. In most cases it is better to use 'tell' together with the sender - * parameter to implement non-blocking request/response message exchanges. - * - * If you are sending messages using ask and using blocking operations on the Future, such as - * 'get', then you have to use getContext().sender().tell(...) - * in the target actor to send a reply message to the original sender, and thereby completing the Future, - * otherwise the sender will block until the timeout expires. - * - * When using future callbacks, inside actors you need to carefully avoid closing over - * the containing actor’s reference, i.e. do not call methods or access mutable state - * on the enclosing actor from within the callback. This would break the actor - * encapsulation and may introduce synchronization bugs and race conditions because - * the callback will be scheduled concurrently to the enclosing actor. Unfortunately - * there is not yet a way to detect these illegal accesses at compile time. - */ - def ask(message: AnyRef, timeout: Timeout): Future[AnyRef] = ?(message, timeout).asInstanceOf[Future[AnyRef]] - - def ask(message: AnyRef, timeoutMillis: Long): Future[AnyRef] = ask(message, new Timeout(timeoutMillis)) - - /** - * Sends a message asynchronously, returning a future which may eventually hold the reply. - * The Future will be completed with an [[akka.actor.AskTimeoutException]] after the given - * timeout has expired. - * - * NOTE: - * Use this method with care. In most cases it is better to use '!' together with implicit or explicit - * sender parameter to implement non-blocking request/response message exchanges. - * - * If you are sending messages using ask and using blocking operations on the Future, such as - * 'get', then you have to use getContext().sender().tell(...) - * in the target actor to send a reply message to the original sender, and thereby completing the Future, - * otherwise the sender will block until the timeout expires. - * - * When using future callbacks, inside actors you need to carefully avoid closing over - * the containing actor’s reference, i.e. do not call methods or access mutable state - * on the enclosing actor from within the callback. This would break the actor - * encapsulation and may introduce synchronization bugs and race conditions because - * the callback will be scheduled concurrently to the enclosing actor. Unfortunately - * there is not yet a way to detect these illegal accesses at compile time. - */ - def ?(message: Any)(implicit timeout: Timeout): Future[Any] = { - implicit val dispatcher = actorRef.asInstanceOf[InternalActorRef].provider.dispatcher - val promise = Promise[Any]() - actorRef.!(message)(promise) - promise - } - - /** - * Sends a message asynchronously, returning a future which may eventually hold the reply. - * The implicit parameter with the default value is just there to disambiguate it from the version that takes the - * implicit timeout - */ - def ?(message: Any, timeout: Timeout)(implicit ignore: Int = 0): Future[Any] = ?(message)(timeout) - } - -} - -object Patterns { - - import akka.actor.ActorRef - import akka.dispatch.Future - import akka.patterns.{ ask ⇒ actorRef2Askable } - import akka.util.Timeout - - def ask(actor: ActorRef, message: Any)(implicit timeout: Timeout): Future[Any] = - actorRef2Askable(actor).?(message) - - def ask(actor: ActorRef, message: Any, timeout: Timeout)(implicit ignore: Int = 0): Future[Any] = - actorRef2Askable(actor).?(message)(timeout) - - def ask(actor: ActorRef, message: Any, timeoutMillis: Long): Future[Any] = - actorRef2Askable(actor).?(message)(new Timeout(timeoutMillis)) - -} diff --git a/akka-actor/src/main/scala/akka/patterns/AskableActorRef.scala b/akka-actor/src/main/scala/akka/patterns/AskableActorRef.scala new file mode 100644 index 0000000000..1b42fc664c --- /dev/null +++ b/akka-actor/src/main/scala/akka/patterns/AskableActorRef.scala @@ -0,0 +1,84 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka.patterns + +import akka.actor.{ ActorRef, InternalActorRef, AskTimeoutException } +import akka.dispatch.{ Future, Promise } +import akka.util.Timeout + +final class AskableActorRef(val actorRef: ActorRef) { + + /** + * 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 + * given timeout has expired; this is independent from any timeout applied + * while awaiting a result for this future (i.e. in + * `Await.result(..., timeout)`). + * + * Warning: + * When using future callbacks, inside actors you need to carefully avoid closing over + * the containing actor’s object, i.e. do not call methods or access mutable state + * on the enclosing actor from within the callback. This would break the actor + * encapsulation and may introduce synchronization bugs and race conditions because + * the callback will be scheduled concurrently to the enclosing actor. Unfortunately + * there is not yet a way to detect these illegal accesses at compile time. + * + * Recommended usage: + * + * {{{ + * val f = worker.ask(request)(timeout) + * flow { + * EnrichedRequest(request, f()) + * } pipeTo nextActor + * }}} + * + * [see the [[akka.dispatch.Future]] companion object for a description of `flow`] + */ + def ask(message: AnyRef)(implicit timeout: Timeout): Future[Any] = akka.patterns.ask(actorRef, message)(timeout) + + /** + * 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 + * given timeout has expired; this is independent from any timeout applied + * while awaiting a result for this future (i.e. in + * `Await.result(..., timeout)`). + * + * Warning: + * When using future callbacks, inside actors you need to carefully avoid closing over + * the containing actor’s object, i.e. do not call methods or access mutable state + * on the enclosing actor from within the callback. This would break the actor + * encapsulation and may introduce synchronization bugs and race conditions because + * the callback will be scheduled concurrently to the enclosing actor. Unfortunately + * there is not yet a way to detect these illegal accesses at compile time. + * + * Recommended usage: + * + * {{{ + * val f = worker ? request + * flow { + * EnrichedRequest(request, f()) + * } pipeTo nextActor + * }}} + * + * [see the [[akka.dispatch.Future]] companion object for a description of `flow`] + */ + def ?(message: Any)(implicit timeout: Timeout): Future[Any] = akka.patterns.ask(actorRef, message) + + /* + * FIXME: I think this should be removed, since it introduces an “ambiguity” + * when sending Tuple2, which the compiler resolves unexpectedly to this + * method; also overloading is bad, isn’t it? - RK (ticket #1653) + */ + /** + * Sends a message asynchronously, returning a future which may eventually hold the reply. + * The implicit parameter with the default value is just there to disambiguate it from the version that takes the + * implicit timeout + */ + def ?(message: Any, timeout: Timeout)(implicit ignore: Int = 0): Future[Any] = ?(message)(timeout) + +} diff --git a/akka-actor/src/main/scala/akka/patterns/package.scala b/akka-actor/src/main/scala/akka/patterns/package.scala new file mode 100644 index 0000000000..d0cd07766a --- /dev/null +++ b/akka-actor/src/main/scala/akka/patterns/package.scala @@ -0,0 +1,70 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka + +package object patterns { + + import akka.actor.{ ActorRef, InternalActorRef, ActorRefWithProvider, AskTimeoutException } + import akka.dispatch.{ Future, Promise } + import akka.util.Timeout + + /** + * Import this implicit conversion to gain `?` and `ask` methods on + * [[akka.actor.ActorRef]], which will defer to the + * `ask(actorRef, message)(timeout)` method defined here. + * + * {{{ + * import akka.patterns.ask + * + * val future = actor ? message // => ask(actor, message) + * val future = actor ask message // => ask(actor, message) + * val future = actor.ask(message)(timeout) // => ask(actor, message)(timeout) + * }}} + * + * All of the above use an implicit [[akka.actor.Timeout]]. + */ + implicit def ask(actorRef: ActorRef): AskableActorRef = new AskableActorRef(actorRef) + + /** + * 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 + * given timeout has expired; this is independent from any timeout applied + * while awaiting a result for this future (i.e. in + * `Await.result(..., timeout)`). + * + * Warning: + * When using future callbacks, inside actors you need to carefully avoid closing over + * the containing actor’s object, i.e. do not call methods or access mutable state + * on the enclosing actor from within the callback. This would break the actor + * encapsulation and may introduce synchronization bugs and race conditions because + * the callback will be scheduled concurrently to the enclosing actor. Unfortunately + * there is not yet a way to detect these illegal accesses at compile time. + * + * Recommended usage: + * + * {{{ + * val f = ask(worker, request)(timeout) + * flow { + * EnrichedRequest(request, f()) + * } pipeTo nextActor + * }}} + * + * [see [[akka.dispatch.Future]] for a description of `flow`] + */ + def ask(actorRef: ActorRef, message: Any)(implicit timeout: Timeout): Future[Any] = actorRef match { + case ref: ActorRefWithProvider ⇒ + ref.provider.ask(timeout) match { + case Some(ref) ⇒ + actorRef.tell(message, ref) + ref.result + case None ⇒ + actorRef.tell(message) + Promise.failed(new AskTimeoutException("failed to create PromiseActorRef"))(ref.provider.dispatcher) + } + case _ ⇒ throw new IllegalArgumentException("incompatible ActorRef " + actorRef) + } + +} diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index 36f5fe9670..11f8fee4af 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -351,10 +351,7 @@ trait BroadcastLike { this: RouterConfig ⇒ createAndRegisterRoutees(props, context, nrOfInstances, routees) { - case (sender, message) ⇒ - message match { - case _ ⇒ toAll(sender, ref.routees) - } + case (sender, message) ⇒ toAll(sender, ref.routees) } } } @@ -407,12 +404,9 @@ trait ScatterGatherFirstCompletedLike { this: RouterConfig ⇒ { case (sender, message) ⇒ val provider: ActorRefProvider = context.asInstanceOf[ActorCell].systemImpl.provider - val promise = Promise[Any]()(provider.dispatcher) - val asker = provider.ask(promise, Timeout(within)).get - promise.pipeTo(sender) - message match { - case _ ⇒ toAll(asker, ref.routees) - } + val asker = provider.ask(Timeout(within)).get + asker.result.pipeTo(sender) + toAll(asker, ref.routees) } } } diff --git a/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala b/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala index f804e29bac..fac3411cde 100644 --- a/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala @@ -8,7 +8,6 @@ import akka.actor.Actor import akka.actor.Props import akka.event.Logging import akka.dispatch.Future -import akka.Patterns //#imports1 @@ -221,6 +220,8 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) { "using ask" in { //#using-ask + import akka.patterns.ask + class MyActor extends Actor { def receive = { case x: String ⇒ sender ! x.toUpperCase @@ -230,10 +231,10 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) { val myActor = system.actorOf(Props(new MyActor), name = "myactor") implicit val timeout = system.settings.ActorTimeout - val future = Patterns.ask(myActor, "hello") + val future = ask(myActor, "hello") for (x ← future) println(x) //Prints "hello" - val result: Future[Int] = for (x ← Patterns.ask(myActor, 3).mapTo[Int]) yield { 2 * x } + val result: Future[Int] = for (x ← ask(myActor, 3).mapTo[Int]) yield { 2 * x } //#using-ask system.stop(myActor) diff --git a/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala b/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala index f65f8224db..5c86a0c295 100644 --- a/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala @@ -13,7 +13,6 @@ import akka.dispatch.Future import akka.dispatch.Await import akka.util.duration._ import akka.dispatch.Promise -import akka.Patterns object FutureDocSpec { @@ -45,9 +44,10 @@ class FutureDocSpec extends AkkaSpec { val msg = "hello" //#ask-blocking import akka.dispatch.Await + import akka.patterns.ask implicit val timeout = system.settings.ActorTimeout - val future = Patterns.ask(actor, msg) + val future = actor ? msg // enabled by the “ask” import val result = Await.result(future, timeout.duration).asInstanceOf[String] //#ask-blocking result must be("HELLO") @@ -59,8 +59,9 @@ class FutureDocSpec extends AkkaSpec { implicit val timeout = system.settings.ActorTimeout //#map-to import akka.dispatch.Future + import akka.patterns.ask - val future: Future[String] = Patterns.ask(actor, msg).mapTo[String] + val future: Future[String] = ask(actor, msg).mapTo[String] //#map-to Await.result(future, timeout.duration) must be("HELLO") } @@ -148,15 +149,16 @@ class FutureDocSpec extends AkkaSpec { val msg2 = 2 implicit val timeout = system.settings.ActorTimeout import akka.dispatch.Await + import akka.patterns.ask //#composing-wrong - val f1 = Patterns.ask(actor1, msg1) - val f2 = Patterns.ask(actor2, msg2) + val f1 = ask(actor1, msg1) + val f2 = ask(actor2, msg2) val a = Await.result(f1, 1 second).asInstanceOf[Int] val b = Await.result(f2, 1 second).asInstanceOf[Int] - val f3 = Patterns.ask(actor3, (a + b)) + val f3 = ask(actor3, (a + b)) val result = Await.result(f3, 1 second).asInstanceOf[Int] //#composing-wrong @@ -171,15 +173,16 @@ class FutureDocSpec extends AkkaSpec { val msg2 = 2 implicit val timeout = system.settings.ActorTimeout import akka.dispatch.Await + import akka.patterns.ask //#composing - val f1 = Patterns.ask(actor1, msg1) - val f2 = Patterns.ask(actor2, msg2) + val f1 = ask(actor1, msg1) + val f2 = ask(actor2, msg2) val f3 = for { a ← f1.mapTo[Int] b ← f2.mapTo[Int] - c ← Patterns.ask(actor3, (a + b)).mapTo[Int] + c ← ask(actor3, (a + b)).mapTo[Int] } yield c val result = Await.result(f3, 1 second).asInstanceOf[Int] @@ -192,7 +195,7 @@ class FutureDocSpec extends AkkaSpec { val oddActor = system.actorOf(Props[OddActor]) //#sequence-ask // oddActor returns odd numbers sequentially from 1 as a List[Future[Int]] - val listOfFutures = List.fill(100)(Patterns.ask(oddActor, GetNext).mapTo[Int]) + val listOfFutures = List.fill(100)(akka.patterns.ask(oddActor, GetNext).mapTo[Int]) // now we have a Future[List[Int]] val futureList = Future.sequence(listOfFutures) @@ -240,7 +243,7 @@ class FutureDocSpec extends AkkaSpec { val actor = system.actorOf(Props[MyActor]) val msg1 = -1 //#recover - val future = Patterns.ask(actor, msg1) recover { + val future = akka.patterns.ask(actor, msg1) recover { case e: ArithmeticException ⇒ 0 } //#recover diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 5b38996d3b..65417d9063 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -138,7 +138,7 @@ class RemoteActorRefProvider( def actorFor(ref: InternalActorRef, path: Iterable[String]): InternalActorRef = local.actorFor(ref, path) - def ask(result: Promise[Any], within: Timeout): Option[ActorRef] = local.ask(result, within) + def ask(within: Timeout): Option[PromiseActorRef] = local.ask(within) /** * Using (checking out) actor on a specific node. @@ -160,12 +160,12 @@ trait RemoteRef extends ActorRefScope { * This reference is network-aware (remembers its origin) and immutable. */ private[akka] class RemoteActorRef private[akka] ( - override val provider: RemoteActorRefProvider, + val provider: RemoteActorRefProvider, remote: RemoteSupport[ParsedTransportAddress], val path: ActorPath, val getParent: InternalActorRef, loader: Option[ClassLoader]) - extends InternalActorRef with RemoteRef { + extends InternalActorRef with RemoteRef with ActorRefWithProvider { def getChild(name: Iterator[String]): InternalActorRef = { val s = name.toStream From cac7c8cdda4cf5eb311fa715d9e8daf3b946456f Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Tue, 17 Jan 2012 23:45:37 -0700 Subject: [PATCH 019/152] Proper fault tolerance in IO test Actors and less blocking in tests --- .../src/test/scala/akka/actor/IOActor.scala | 160 +++++++++--------- 1 file changed, 82 insertions(+), 78 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala index a7f06d9eb2..676bb2c4a8 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala @@ -14,13 +14,13 @@ object IOActorSpec { class SimpleEchoServer(host: String, port: Int) extends Actor { - IOManager(context.system) listen (host, port) + val server = IOManager(context.system) listen (host, port) val state = IO.IterateeRef.Map.sync[IO.Handle]() def receive = { - case IO.NewClient(server) ⇒ + case IO.NewClient(`server`) ⇒ val socket = server.accept() state(socket) flatMap (_ ⇒ IO repeat (IO.takeAny map socket.write)) @@ -32,11 +32,15 @@ object IOActorSpec { } + override def postStop { + server.close() + state.keySet foreach (_.close()) + } } class SimpleEchoClient(host: String, port: Int) extends Actor { - var socket = IOManager(context.system) connect (host, port) + val socket = IOManager(context.system) connect (host, port) val state = IO.IterateeRef.sync() @@ -51,16 +55,21 @@ object IOActorSpec { } } - case IO.Read(_, bytes) ⇒ + case IO.Read(`socket`, bytes) ⇒ state(IO Chunk bytes) - case IO.Connected(_) ⇒ + case IO.Connected(`socket`) ⇒ - case IO.Closed(_, cause) ⇒ + case IO.Closed(`socket`, cause) ⇒ state(IO EOF cause) - socket = IOManager(context.system) connect (host, port) + throw (cause getOrElse new RuntimeException("Socket closed")) } + + override def postStop { + socket.close() + state(IO EOF None) + } } sealed trait KVCommand { @@ -80,7 +89,7 @@ object IOActorSpec { } // Basic Redis-style protocol - class KVStore(host: String, port: Int, started: TestLatch) extends Actor { + class KVStore(host: String, port: Int) extends Actor { import context.system @@ -88,15 +97,13 @@ object IOActorSpec { var kvs: Map[String, String] = Map.empty - IOManager(context.system) listen (host, port) - - started.open + val server = IOManager(context.system) listen (host, port) val EOL = ByteString("\r\n") def receive = { - case IO.NewClient(server) ⇒ + case IO.NewClient(`server`) ⇒ val socket = server.accept() state(socket) flatMap { _ ⇒ IO repeat { @@ -144,6 +151,10 @@ object IOActorSpec { } + override def postStop { + server.close() + state.keySet foreach (_.close()) + } } class KVClient(host: String, port: Int) extends Actor { @@ -158,46 +169,54 @@ object IOActorSpec { case cmd: KVCommand ⇒ val source = sender socket write cmd.bytes - for { - _ ← state - result ← readResult - } yield result.fold(err ⇒ source ! Status.Failure(new RuntimeException(err)), source !) + state flatMap { _ ⇒ + readResult map (source !) recover { + case e ⇒ source ! Status.Failure(e) + } + } - case IO.Read(socket, bytes) ⇒ + case IO.Read(`socket`, bytes) ⇒ state(IO Chunk bytes) - case IO.Connected(socket) ⇒ + case IO.Connected(`socket`) ⇒ - case IO.Closed(socket, cause) ⇒ + case IO.Closed(`socket`, cause) ⇒ + state(IO EOF cause) + throw (cause getOrElse new RuntimeException("Socket closed")) } - def readResult: IO.Iteratee[Either[String, Any]] = { + override def postStop { + socket.close() + state(IO EOF None) + } + + def readResult: IO.Iteratee[Any] = { IO take 1 map (_.utf8String) flatMap { - case "+" ⇒ IO takeUntil EOL map (msg ⇒ Right(msg.utf8String)) - case "-" ⇒ IO takeUntil EOL map (err ⇒ Left(err.utf8String)) + case "+" ⇒ IO takeUntil EOL map (msg ⇒ msg.utf8String) + case "-" ⇒ IO takeUntil EOL flatMap (err ⇒ IO throwErr new RuntimeException(err.utf8String)) case "$" ⇒ IO takeUntil EOL map (_.utf8String.toInt) flatMap { - case -1 ⇒ IO Iteratee Right(None) + case -1 ⇒ IO Done None case length ⇒ for { value ← IO take length _ ← IO takeUntil EOL - } yield Right(Some(value.utf8String)) + } yield Some(value.utf8String) } case "*" ⇒ IO takeUntil EOL map (_.utf8String.toInt) flatMap { - case -1 ⇒ IO Iteratee Right(None) + case -1 ⇒ IO Done None case length ⇒ - IO.takeList(length)(readResult) map { list ⇒ + IO.takeList(length)(readResult) flatMap { list ⇒ ((Right(Map()): Either[String, Map[String, String]]) /: list.grouped(2)) { - case (Right(m), List(Right(Some(k: String)), Right(Some(v: String)))) ⇒ Right(m + (k -> v)) + case (Right(m), List(Some(k: String), Some(v: String))) ⇒ Right(m + (k -> v)) case (Right(_), _) ⇒ Left("Unexpected Response") case (left, _) ⇒ left - } + } fold (msg ⇒ IO throwErr new RuntimeException(msg), IO Done _) } } - case _ ⇒ IO Iteratee Left("Unexpected Response") + case _ ⇒ IO throwErr new RuntimeException("Unexpected Response") } } } @@ -249,61 +268,46 @@ class IOActorSpec extends AkkaSpec with DefaultTimeout { "an IO Actor" must { "run echo server" in { - val client = system.actorOf(Props(new SimpleEchoClient("localhost", 8064))) - val f1 = retry() { client ? ByteString("Hello World!1") } - val f2 = retry() { client ? ByteString("Hello World!2") } - val f3 = retry() { client ? ByteString("Hello World!3") } - val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8064))) - Await.result(f1, TestLatch.DefaultTimeout) must equal(ByteString("Hello World!1")) - Await.result(f2, TestLatch.DefaultTimeout) must equal(ByteString("Hello World!2")) - Await.result(f3, TestLatch.DefaultTimeout) must equal(ByteString("Hello World!3")) + filterException[java.net.ConnectException] { + val client = system.actorOf(Props(new SimpleEchoClient("localhost", 8064))) + val f1 = retry() { client ? ByteString("Hello World!1") } + val f2 = retry() { client ? ByteString("Hello World!2") } + val f3 = retry() { client ? ByteString("Hello World!3") } + val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8064))) + Await.result(f1, TestLatch.DefaultTimeout) must equal(ByteString("Hello World!1")) + Await.result(f2, TestLatch.DefaultTimeout) must equal(ByteString("Hello World!2")) + Await.result(f3, TestLatch.DefaultTimeout) must equal(ByteString("Hello World!3")) + } } "run echo server under high load" in { - val client = system.actorOf(Props(new SimpleEchoClient("localhost", 8065))) - val list = List.range(0, 1000) - val f = Future.traverse(list)(i ⇒ retry() { client ? ByteString(i.toString) }) - val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8065))) - assert(Await.result(f, TestLatch.DefaultTimeout).size === 1000) + filterException[java.net.ConnectException] { + val client = system.actorOf(Props(new SimpleEchoClient("localhost", 8065))) + val list = List.range(0, 1000) + val f = Future.traverse(list)(i ⇒ retry() { client ? ByteString(i.toString) }) + val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8065))) + assert(Await.result(f, TestLatch.DefaultTimeout).size === 1000) + } } - // Not currently configurable at runtime - /* - "run echo server under high load with small buffer" in { - val started = TestLatch(1) - val ioManager = actorOf(new IOManager(2)) - val server = actorOf(new SimpleEchoServer("localhost", 8066, ioManager, started)) - started.await - val client = actorOf(new SimpleEchoClient("localhost", 8066, ioManager)) - val list = List.range(0, 1000) - val f = Future.traverse(list)(i ⇒ client ? ByteString(i.toString)) - assert(Await.result(f, timeout.duration).size === 1000) - system.stop(client) - system.stop(server) - system.stop(ioManager) - } - */ - "run key-value store" in { - val started = TestLatch(1) - val server = system.actorOf(Props(new KVStore("localhost", 8067, started))) - Await.ready(started, TestLatch.DefaultTimeout) - val client1 = system.actorOf(Props(new KVClient("localhost", 8067))) - val client2 = system.actorOf(Props(new KVClient("localhost", 8067))) - val f1 = client1 ? KVSet("hello", "World") - val f2 = client1 ? KVSet("test", "No one will read me") - val f3 = client1 ? KVGet("hello") - Await.ready(f2, TestLatch.DefaultTimeout) - val f4 = client2 ? KVSet("test", "I'm a test!") - Await.ready(f4, TestLatch.DefaultTimeout) - val f5 = client1 ? KVGet("test") - val f6 = client2 ? KVGetAll - Await.result(f1, TestLatch.DefaultTimeout) must equal("OK") - Await.result(f2, TestLatch.DefaultTimeout) must equal("OK") - Await.result(f3, TestLatch.DefaultTimeout) must equal(Some("World")) - Await.result(f4, TestLatch.DefaultTimeout) must equal("OK") - Await.result(f5, TestLatch.DefaultTimeout) must equal(Some("I'm a test!")) - Await.result(f6, TestLatch.DefaultTimeout) must equal(Map("hello" -> "World", "test" -> "I'm a test!")) + filterException[java.net.ConnectException] { + val client1 = system.actorOf(Props(new KVClient("localhost", 8067))) + val client2 = system.actorOf(Props(new KVClient("localhost", 8067))) + val f1 = retry() { client1 ? KVSet("hello", "World") } + val f2 = retry() { client1 ? KVSet("test", "No one will read me") } + val f3 = f1 flatMap { _ ⇒ retry() { client1 ? KVGet("hello") } } + val f4 = f2 flatMap { _ ⇒ retry() { client2 ? KVSet("test", "I'm a test!") } } + val f5 = f4 flatMap { _ ⇒ retry() { client1 ? KVGet("test") } } + val f6 = Future.sequence(List(f3, f5)) flatMap { _ ⇒ retry() { client2 ? KVGetAll } } + val server = system.actorOf(Props(new KVStore("localhost", 8067))) + Await.result(f1, TestLatch.DefaultTimeout) must equal("OK") + Await.result(f2, TestLatch.DefaultTimeout) must equal("OK") + Await.result(f3, TestLatch.DefaultTimeout) must equal(Some("World")) + Await.result(f4, TestLatch.DefaultTimeout) must equal("OK") + Await.result(f5, TestLatch.DefaultTimeout) must equal(Some("I'm a test!")) + Await.result(f6, TestLatch.DefaultTimeout) must equal(Map("hello" -> "World", "test" -> "I'm a test!")) + } } } From 2bed2cb9547c235e44cccbc69f82411004b8451c Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 18 Jan 2012 10:18:51 +0100 Subject: [PATCH 020/152] move everything into package akka.pattern --- .../akka/actor/ActorFireForgetRequestReplySpec.scala | 2 +- .../test/scala/akka/actor/ActorLifeCycleSpec.scala | 2 +- .../src/test/scala/akka/actor/ActorLookupSpec.scala | 2 +- .../src/test/scala/akka/actor/ActorRefSpec.scala | 2 +- .../src/test/scala/akka/actor/ActorTimeoutSpec.scala | 2 +- .../src/test/scala/akka/actor/DeathWatchSpec.scala | 2 +- .../src/test/scala/akka/actor/ForwardActorSpec.scala | 2 +- .../src/test/scala/akka/actor/IOActor.scala | 2 +- .../test/scala/akka/actor/RestartStrategySpec.scala | 2 +- .../src/test/scala/akka/actor/SchedulerSpec.scala | 2 +- .../scala/akka/actor/SupervisorHierarchySpec.scala | 2 +- .../test/scala/akka/actor/SupervisorMiscSpec.scala | 2 +- .../src/test/scala/akka/actor/SupervisorSpec.scala | 2 +- .../test/scala/akka/actor/SupervisorTreeSpec.scala | 2 +- .../src/test/scala/akka/actor/Ticket669Spec.scala | 2 +- .../src/test/scala/akka/actor/TypedActorSpec.scala | 2 +- .../scala/akka/actor/dispatch/ActorModelSpec.scala | 2 +- .../akka/actor/dispatch/DispatcherActorSpec.scala | 2 +- .../scala/akka/actor/dispatch/PinnedActorSpec.scala | 2 +- .../src/test/scala/akka/dataflow/Future2Actor.scala | 2 +- .../src/test/scala/akka/dispatch/FutureSpec.scala | 2 +- .../scala/akka/dispatch/PriorityDispatcherSpec.scala | 2 +- .../src/test/scala/akka/routing/ActorPoolSpec.scala | 2 +- .../akka/routing/ConfiguredLocalRoutingSpec.scala | 2 +- .../src/test/scala/akka/routing/RoutingSpec.scala | 2 +- .../scala/akka/serialization/SerializeSpec.scala | 2 +- .../src/test/scala/akka/ticket/Ticket703Spec.scala | 2 +- akka-actor/src/main/scala/akka/actor/ActorRef.scala | 4 ++-- .../src/main/scala/akka/actor/ActorSystem.scala | 2 +- .../src/main/scala/akka/actor/TypedActor.scala | 2 +- akka-actor/src/main/scala/akka/event/Logging.scala | 2 +- .../akka/{patterns => pattern}/AskableActorRef.scala | 6 +++--- .../src/main/scala/akka/{ => pattern}/Patterns.scala | 6 +++--- .../scala/akka/{patterns => pattern}/package.scala | 4 ++-- akka-agent/src/main/scala/akka/agent/Agent.scala | 2 +- .../code/akka/docs/actor/FaultHandlingTestBase.java | 2 +- .../akka/docs/actor/UntypedActorDocTestBase.java | 2 +- .../code/akka/docs/future/FutureDocTestBase.java | 2 +- .../code/akka/docs/transactor/TransactorDocTest.java | 2 +- .../scala/code/akka/docs/actor/ActorDocSpec.scala | 6 +++--- .../scala/code/akka/docs/future/FutureDocSpec.scala | 12 ++++++------ .../code/akka/docs/routing/RouterTypeExample.scala | 2 +- .../code/akka/docs/testkit/TestkitDocSpec.scala | 4 ++-- .../akka/docs/transactor/TransactorDocSpec.scala | 2 +- .../src/main/scala/akka/remote/Gossiper.scala | 2 +- .../remote/DirectRoutedRemoteActorMultiJvmSpec.scala | 2 +- .../akka/remote/NewRemoteActorMultiJvmSpec.scala | 2 +- .../remote/RandomRoutedRemoteActorMultiJvmSpec.scala | 2 +- .../RoundRobinRoutedRemoteActorMultiJvmSpec.scala | 2 +- .../scala/akka/remote/RemoteCommunicationSpec.scala | 2 +- .../src/main/scala/akka/testkit/TestActorRef.scala | 2 +- .../src/test/scala/akka/testkit/AkkaSpec.scala | 2 +- .../test/scala/akka/testkit/TestActorRefSpec.scala | 2 +- .../src/test/scala/akka/testkit/TestProbeSpec.scala | 2 +- .../transactor/UntypedCoordinatedIncrementTest.java | 2 +- .../java/akka/transactor/UntypedTransactorTest.java | 2 +- .../akka/transactor/CoordinatedIncrementSpec.scala | 2 +- .../scala/akka/transactor/FickleFriendsSpec.scala | 2 +- .../test/scala/akka/transactor/TransactorSpec.scala | 2 +- 59 files changed, 73 insertions(+), 73 deletions(-) rename akka-actor/src/main/scala/akka/{patterns => pattern}/AskableActorRef.scala (96%) rename akka-actor/src/main/scala/akka/{ => pattern}/Patterns.scala (97%) rename akka-actor/src/main/scala/akka/{patterns => pattern}/package.scala (97%) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala index 8a4d643e29..4665c8dbf6 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala @@ -8,7 +8,7 @@ import akka.testkit._ import org.scalatest.BeforeAndAfterEach import akka.util.duration._ import akka.dispatch.Await -import akka.patterns.ask +import akka.pattern.ask object ActorFireForgetRequestReplySpec { diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala index 016b738c45..e85dffa45b 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala @@ -12,7 +12,7 @@ import akka.testkit._ import akka.util.duration._ import java.util.concurrent.atomic._ import akka.dispatch.Await -import akka.patterns.ask +import akka.pattern.ask object ActorLifeCycleSpec { diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala index 8a990e18ca..135a7efe0a 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala @@ -6,7 +6,7 @@ package akka.actor import akka.testkit._ import akka.util.duration._ import akka.dispatch.Await -import akka.patterns.ask +import akka.pattern.ask object ActorLookupSpec { diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala index a78a99a79b..8dbd1c9a8c 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala @@ -15,7 +15,7 @@ import akka.util.ReflectiveAccess import akka.serialization.Serialization import java.util.concurrent.{ CountDownLatch, TimeUnit } import akka.dispatch.{ Await, DefaultPromise, Promise, Future } -import akka.patterns.ask +import akka.pattern.ask object ActorRefSpec { diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala index 1c9a7e67d6..e5b2303140 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala @@ -10,7 +10,7 @@ import akka.testkit.DefaultTimeout import java.util.concurrent.TimeoutException import akka.dispatch.Await import akka.util.Timeout -import akka.patterns.ask +import akka.pattern.ask @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class ActorTimeoutSpec extends AkkaSpec with BeforeAndAfterAll with DefaultTimeout { diff --git a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala index 1a9c84ecd9..b704991203 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala @@ -8,7 +8,7 @@ import akka.testkit._ import akka.util.duration._ import java.util.concurrent.atomic._ import akka.dispatch.Await -import akka.patterns.ask +import akka.pattern.ask @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class LocalDeathWatchSpec extends AkkaSpec with ImplicitSender with DefaultTimeout with DeathWatchSpec diff --git a/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala index a019b4afb3..95ecfe1280 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala @@ -9,7 +9,7 @@ import akka.util.duration._ import Actor._ import akka.util.Duration import akka.dispatch.Await -import akka.patterns.ask +import akka.pattern.ask object ForwardActorSpec { val ExpectedMessage = "FOO" diff --git a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala index aaa86e9c3e..0d70da3649 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala @@ -11,7 +11,7 @@ import akka.util.cps._ import scala.util.continuations._ import akka.testkit._ import akka.dispatch.{ Await, Future } -import akka.patterns.ask +import akka.pattern.ask object IOActorSpec { import IO._ diff --git a/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala index 0730b8de7c..49d0359e16 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala @@ -14,7 +14,7 @@ import akka.testkit.AkkaSpec import akka.testkit.DefaultTimeout import akka.testkit.TestLatch import akka.util.duration._ -import akka.patterns.ask +import akka.pattern.ask @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class RestartStrategySpec extends AkkaSpec with DefaultTimeout { diff --git a/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala index 1269558cf9..6d0bde2299 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala @@ -8,7 +8,7 @@ import java.util.concurrent.{ CountDownLatch, ConcurrentLinkedQueue, TimeUnit } import akka.testkit.DefaultTimeout import akka.testkit.TestLatch import akka.dispatch.Await -import akka.patterns.ask +import akka.pattern.ask @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout { diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala index 88139f97f8..883f771547 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala @@ -8,7 +8,7 @@ import akka.testkit._ import java.util.concurrent.{ TimeUnit, CountDownLatch } import akka.dispatch.Await -import akka.patterns.ask +import akka.pattern.ask object SupervisorHierarchySpec { class FireWorkerException(msg: String) extends Exception(msg) diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala index 8b32f76e49..319521fa1f 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala @@ -8,7 +8,7 @@ import akka.dispatch.{ PinnedDispatcher, Dispatchers, Await } import java.util.concurrent.{ TimeUnit, CountDownLatch } import akka.testkit.AkkaSpec import akka.testkit.DefaultTimeout -import akka.patterns.ask +import akka.pattern.ask object SupervisorMiscSpec { val config = """ diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala index 8405e29e8e..90b3b2d760 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala @@ -11,7 +11,7 @@ import akka.testkit.TestEvent._ import akka.testkit._ import java.util.concurrent.atomic.AtomicInteger import akka.dispatch.Await -import akka.patterns.ask +import akka.pattern.ask object SupervisorSpec { val Timeout = 5 seconds diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala index 6ebca690a7..8c8284ec2a 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala @@ -12,7 +12,7 @@ import akka.testkit.AkkaSpec import akka.testkit.ImplicitSender import akka.testkit.DefaultTimeout import akka.dispatch.{ Await, Dispatchers } -import akka.patterns.ask +import akka.pattern.ask @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class SupervisorTreeSpec extends AkkaSpec with ImplicitSender with DefaultTimeout { diff --git a/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala b/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala index f3a0784da2..54aa978ba9 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala @@ -11,7 +11,7 @@ import akka.testkit.AkkaSpec import akka.testkit.ImplicitSender import akka.testkit.DefaultTimeout import akka.dispatch.Await -import akka.patterns.ask +import akka.pattern.ask @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class Ticket669Spec extends AkkaSpec with BeforeAndAfterAll with ImplicitSender with DefaultTimeout { diff --git a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala index 7cde57720c..28e8fb8d8f 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala @@ -18,7 +18,7 @@ import java.util.concurrent.{ TimeUnit, CountDownLatch } import akka.japi.{ Creator, Option ⇒ JOption } import akka.testkit.DefaultTimeout import akka.dispatch.{ Await, Dispatchers, Future, Promise } -import akka.patterns.ask +import akka.pattern.ask object TypedActorSpec { diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala index c242617753..c9cddba4b3 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala @@ -20,7 +20,7 @@ import akka.util.duration._ import akka.event.Logging.Error import com.typesafe.config.Config import akka.util.Duration -import akka.patterns.ask +import akka.pattern.ask object ActorModelSpec { diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala index 1160883e09..2dce8346db 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala @@ -8,7 +8,7 @@ import akka.util.Duration import akka.util.duration._ import akka.testkit.DefaultTimeout import akka.dispatch.{ Await, PinnedDispatcher, Dispatchers, Dispatcher } -import akka.patterns.ask +import akka.pattern.ask object DispatcherActorSpec { val config = """ diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala index 4ddcb8ba12..6c66784e5d 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala @@ -7,7 +7,7 @@ import akka.actor.{ Props, Actor } import akka.testkit.AkkaSpec import org.scalatest.BeforeAndAfterEach import akka.dispatch.{ Await, PinnedDispatcher, Dispatchers } -import akka.patterns.ask +import akka.pattern.ask object PinnedActorSpec { val config = """ diff --git a/akka-actor-tests/src/test/scala/akka/dataflow/Future2Actor.scala b/akka-actor-tests/src/test/scala/akka/dataflow/Future2Actor.scala index 48992a5ee7..91cc9b2c93 100644 --- a/akka-actor-tests/src/test/scala/akka/dataflow/Future2Actor.scala +++ b/akka-actor-tests/src/test/scala/akka/dataflow/Future2Actor.scala @@ -9,7 +9,7 @@ import akka.actor.future2actor import akka.util.duration._ import akka.testkit.AkkaSpec import akka.testkit.DefaultTimeout -import akka.patterns.ask +import akka.pattern.ask class Future2ActorSpec extends AkkaSpec with DefaultTimeout { diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala index c874ea68b7..face8580f2 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala @@ -16,7 +16,7 @@ import akka.testkit.DefaultTimeout import akka.testkit.TestLatch import java.util.concurrent.{ TimeoutException, TimeUnit, CountDownLatch } import scala.runtime.NonLocalReturnControl -import akka.patterns.ask +import akka.pattern.ask object FutureSpec { class TestActor extends Actor { diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala index 82fcc372fa..2bb4ab73e5 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala @@ -2,7 +2,7 @@ package akka.dispatch import akka.actor.{ Props, LocalActorRef, Actor } import akka.testkit.AkkaSpec -import akka.patterns.ask +import akka.pattern.ask import akka.util.duration._ import akka.testkit.DefaultTimeout import com.typesafe.config.Config diff --git a/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala index bfdb168156..5d6cc16eda 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala @@ -6,7 +6,7 @@ import akka.util.duration._ import java.util.concurrent.atomic.{ AtomicBoolean, AtomicInteger } import akka.testkit.AkkaSpec import akka.dispatch.{ Await, Promise, Future } -import akka.patterns.ask +import akka.pattern.ask object ActorPoolSpec { diff --git a/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala index 2d4a4be1c2..3cc1d6893f 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala @@ -6,7 +6,7 @@ import java.util.concurrent.atomic.AtomicInteger import akka.testkit._ import akka.util.duration._ import akka.dispatch.Await -import akka.patterns.ask +import akka.pattern.ask @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class ConfiguredLocalRoutingSpec extends AkkaSpec with DefaultTimeout with ImplicitSender { diff --git a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala index dff790f50f..ad5afa4383 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala @@ -12,7 +12,7 @@ import akka.dispatch.Await import akka.util.Duration import akka.config.ConfigurationException import com.typesafe.config.ConfigFactory -import akka.patterns.ask +import akka.pattern.ask object RoutingSpec { diff --git a/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala b/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala index cfec55413d..02e2a30865 100644 --- a/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala @@ -13,7 +13,7 @@ import akka.util.Timeout import akka.util.duration._ import scala.reflect.BeanInfo import com.google.protobuf.Message -import akka.patterns.ask +import akka.pattern.ask class ProtobufSerializer extends Serializer { val ARRAY_OF_BYTE_ARRAY = Array[Class[_]](classOf[Array[Byte]]) diff --git a/akka-actor-tests/src/test/scala/akka/ticket/Ticket703Spec.scala b/akka-actor-tests/src/test/scala/akka/ticket/Ticket703Spec.scala index 63367ff508..7357de4802 100644 --- a/akka-actor-tests/src/test/scala/akka/ticket/Ticket703Spec.scala +++ b/akka-actor-tests/src/test/scala/akka/ticket/Ticket703Spec.scala @@ -5,7 +5,7 @@ import akka.routing._ import akka.testkit.AkkaSpec import akka.dispatch.Await import akka.util.duration._ -import akka.patterns.ask +import akka.pattern.ask @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class Ticket703Spec extends AkkaSpec { diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index e7c03e8ae0..9ebae64e62 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -168,7 +168,7 @@ trait LocalRef extends ActorRefScope { } /** - * Trait for matching on ActorRefs which have access to a provider; this is used in akka.patterns.ask. + * Trait for matching on ActorRefs which have access to a provider; this is used in akka.pattern.ask. */ trait ActorRefWithProvider { this: InternalActorRef ⇒ def provider: ActorRefProvider @@ -472,7 +472,7 @@ class AskTimeoutException(message: String, cause: Throwable) extends TimeoutExce } /** - * Akka private optimized representation of the temporary actor spawned to + * Akka private optimized representation of the temporary actor spawned to * receive the reply to an "ask" operation. */ private[akka] final class PromiseActorRef( diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 098099c1d1..68a6eb6dc8 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -7,7 +7,7 @@ import akka.config.ConfigurationException import akka.actor._ import akka.event._ import akka.dispatch._ -import akka.patterns.ask +import akka.pattern.ask import akka.util.duration._ import akka.util.Timeout import akka.util.Timeout._ diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index 87fbd98f47..cc0972c56e 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -396,7 +396,7 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi case "equals" ⇒ (args.length == 1 && (proxy eq args(0)) || actor == extension.getActorRefFor(args(0))).asInstanceOf[AnyRef] //Force boxing of the boolean case "hashCode" ⇒ actor.hashCode.asInstanceOf[AnyRef] case _ ⇒ - import akka.patterns.ask + import akka.pattern.ask MethodCall(method, args) match { case m if m.isOneWay ⇒ actor ! m; null //Null return value case m if m.returnsFuture_? ⇒ actor.?(m, timeout) diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index bb0f881c94..6fcf116f80 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -158,7 +158,7 @@ trait LoggingBus extends ActorEventBus { val name = "log" + Extension(system).id() + "-" + simpleName(clazz) val actor = system.systemActorOf(Props(clazz), name) implicit val timeout = Timeout(3 seconds) - import akka.patterns.ask + import akka.pattern.ask val response = try Await.result(actor ? InitializeLogger(this), timeout.duration) catch { case _: TimeoutException ⇒ publish(Warning(simpleName(this), "Logger " + name + " did not respond within " + timeout + " to InitializeLogger(bus)")) diff --git a/akka-actor/src/main/scala/akka/patterns/AskableActorRef.scala b/akka-actor/src/main/scala/akka/pattern/AskableActorRef.scala similarity index 96% rename from akka-actor/src/main/scala/akka/patterns/AskableActorRef.scala rename to akka-actor/src/main/scala/akka/pattern/AskableActorRef.scala index 1b42fc664c..faa4c4d5c0 100644 --- a/akka-actor/src/main/scala/akka/patterns/AskableActorRef.scala +++ b/akka-actor/src/main/scala/akka/pattern/AskableActorRef.scala @@ -1,7 +1,7 @@ /** * Copyright (C) 2009-2011 Typesafe Inc. */ -package akka.patterns +package akka.pattern import akka.actor.{ ActorRef, InternalActorRef, AskTimeoutException } import akka.dispatch.{ Future, Promise } @@ -37,7 +37,7 @@ final class AskableActorRef(val actorRef: ActorRef) { * * [see the [[akka.dispatch.Future]] companion object for a description of `flow`] */ - def ask(message: AnyRef)(implicit timeout: Timeout): Future[Any] = akka.patterns.ask(actorRef, message)(timeout) + def ask(message: AnyRef)(implicit timeout: Timeout): Future[Any] = akka.pattern.ask(actorRef, message)(timeout) /** * Sends a message asynchronously and returns a [[akka.dispatch.Future]] @@ -67,7 +67,7 @@ final class AskableActorRef(val actorRef: ActorRef) { * * [see the [[akka.dispatch.Future]] companion object for a description of `flow`] */ - def ?(message: Any)(implicit timeout: Timeout): Future[Any] = akka.patterns.ask(actorRef, message) + def ?(message: Any)(implicit timeout: Timeout): Future[Any] = akka.pattern.ask(actorRef, message) /* * FIXME: I think this should be removed, since it introduces an “ambiguity” diff --git a/akka-actor/src/main/scala/akka/Patterns.scala b/akka-actor/src/main/scala/akka/pattern/Patterns.scala similarity index 97% rename from akka-actor/src/main/scala/akka/Patterns.scala rename to akka-actor/src/main/scala/akka/pattern/Patterns.scala index 7650c4077f..bc86877f40 100644 --- a/akka-actor/src/main/scala/akka/Patterns.scala +++ b/akka-actor/src/main/scala/akka/pattern/Patterns.scala @@ -1,12 +1,12 @@ /** * Copyright (C) 2009-2011 Typesafe Inc. */ -package akka +package akka.pattern object Patterns { import akka.actor.ActorRef import akka.dispatch.Future - import akka.patterns.{ ask ⇒ scalaAsk } + import akka.pattern.{ ask ⇒ scalaAsk } import akka.util.Timeout /** @@ -68,4 +68,4 @@ object Patterns { * }}} */ def ask(actor: ActorRef, message: Any, timeoutMillis: Long): Future[AnyRef] = scalaAsk(actor, message)(new Timeout(timeoutMillis)).asInstanceOf[Future[AnyRef]] -} \ No newline at end of file +} diff --git a/akka-actor/src/main/scala/akka/patterns/package.scala b/akka-actor/src/main/scala/akka/pattern/package.scala similarity index 97% rename from akka-actor/src/main/scala/akka/patterns/package.scala rename to akka-actor/src/main/scala/akka/pattern/package.scala index d0cd07766a..bfd6384c21 100644 --- a/akka-actor/src/main/scala/akka/patterns/package.scala +++ b/akka-actor/src/main/scala/akka/pattern/package.scala @@ -3,7 +3,7 @@ */ package akka -package object patterns { +package object pattern { import akka.actor.{ ActorRef, InternalActorRef, ActorRefWithProvider, AskTimeoutException } import akka.dispatch.{ Future, Promise } @@ -15,7 +15,7 @@ package object patterns { * `ask(actorRef, message)(timeout)` method defined here. * * {{{ - * import akka.patterns.ask + * import akka.pattern.ask * * val future = actor ? message // => ask(actor, message) * val future = actor ask message // => ask(actor, message) diff --git a/akka-agent/src/main/scala/akka/agent/Agent.scala b/akka-agent/src/main/scala/akka/agent/Agent.scala index 8337c0f64b..cd19a0c781 100644 --- a/akka-agent/src/main/scala/akka/agent/Agent.scala +++ b/akka-agent/src/main/scala/akka/agent/Agent.scala @@ -7,7 +7,7 @@ package akka.agent import akka.actor._ import akka.japi.{ Function ⇒ JFunc, Procedure ⇒ JProc } import akka.dispatch._ -import akka.patterns.ask +import akka.pattern.ask import akka.util.Timeout import scala.concurrent.stm._ diff --git a/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java b/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java index f67347b2af..507b812baa 100644 --- a/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java +++ b/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java @@ -13,7 +13,7 @@ import akka.actor.Props; import akka.actor.Terminated; import akka.actor.UntypedActor; import akka.dispatch.Await; -import static akka.Patterns.ask; +import static akka.pattern.Patterns.ask; import akka.util.Duration; import akka.testkit.AkkaSpec; import akka.testkit.TestProbe; diff --git a/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java b/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java index 4944f817f5..fb6aa72d2f 100644 --- a/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java +++ b/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java @@ -38,7 +38,7 @@ import org.junit.Test; import scala.Option; import java.lang.Object; import java.util.concurrent.TimeUnit; -import akka.Patterns; +import akka.pattern.Patterns; import static org.junit.Assert.*; diff --git a/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java b/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java index dd10afd165..37f0436b27 100644 --- a/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java +++ b/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java @@ -55,7 +55,7 @@ import akka.actor.UntypedActor; import akka.actor.ActorRef; import akka.actor.Props; import akka.dispatch.Futures; -import akka.Patterns; +import akka.pattern.Patterns; import static org.junit.Assert.*; diff --git a/akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java b/akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java index bb2f069dd0..38b60d4888 100644 --- a/akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java +++ b/akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java @@ -10,7 +10,7 @@ import org.junit.Test; //#imports import akka.actor.*; import akka.dispatch.Await; -import static akka.Patterns.ask; +import static akka.pattern.Patterns.ask; import akka.transactor.Coordinated; import akka.util.Duration; import akka.util.Timeout; diff --git a/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala b/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala index fac3411cde..7cf551081b 100644 --- a/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala @@ -220,7 +220,7 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) { "using ask" in { //#using-ask - import akka.patterns.ask + import akka.pattern.ask class MyActor extends Actor { def receive = { @@ -245,7 +245,7 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) { //#using-implicit-timeout import akka.util.duration._ import akka.util.Timeout - import akka.patterns.ask + import akka.pattern.ask implicit val timeout = Timeout(500 millis) val future = myActor ? "hello" //#using-implicit-timeout @@ -257,7 +257,7 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) { val myActor = system.actorOf(Props(new FirstActor)) //#using-explicit-timeout import akka.util.duration._ - import akka.patterns.ask + import akka.pattern.ask val future = myActor ? ("hello", timeout = 500 millis) //#using-explicit-timeout Await.result(future, 500 millis) must be("hello") diff --git a/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala b/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala index 5c86a0c295..cdf866336a 100644 --- a/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala @@ -44,7 +44,7 @@ class FutureDocSpec extends AkkaSpec { val msg = "hello" //#ask-blocking import akka.dispatch.Await - import akka.patterns.ask + import akka.pattern.ask implicit val timeout = system.settings.ActorTimeout val future = actor ? msg // enabled by the “ask” import @@ -59,7 +59,7 @@ class FutureDocSpec extends AkkaSpec { implicit val timeout = system.settings.ActorTimeout //#map-to import akka.dispatch.Future - import akka.patterns.ask + import akka.pattern.ask val future: Future[String] = ask(actor, msg).mapTo[String] //#map-to @@ -149,7 +149,7 @@ class FutureDocSpec extends AkkaSpec { val msg2 = 2 implicit val timeout = system.settings.ActorTimeout import akka.dispatch.Await - import akka.patterns.ask + import akka.pattern.ask //#composing-wrong val f1 = ask(actor1, msg1) @@ -173,7 +173,7 @@ class FutureDocSpec extends AkkaSpec { val msg2 = 2 implicit val timeout = system.settings.ActorTimeout import akka.dispatch.Await - import akka.patterns.ask + import akka.pattern.ask //#composing val f1 = ask(actor1, msg1) @@ -195,7 +195,7 @@ class FutureDocSpec extends AkkaSpec { val oddActor = system.actorOf(Props[OddActor]) //#sequence-ask // oddActor returns odd numbers sequentially from 1 as a List[Future[Int]] - val listOfFutures = List.fill(100)(akka.patterns.ask(oddActor, GetNext).mapTo[Int]) + val listOfFutures = List.fill(100)(akka.pattern.ask(oddActor, GetNext).mapTo[Int]) // now we have a Future[List[Int]] val futureList = Future.sequence(listOfFutures) @@ -243,7 +243,7 @@ class FutureDocSpec extends AkkaSpec { val actor = system.actorOf(Props[MyActor]) val msg1 = -1 //#recover - val future = akka.patterns.ask(actor, msg1) recover { + val future = akka.pattern.ask(actor, msg1) recover { case e: ArithmeticException ⇒ 0 } //#recover diff --git a/akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala b/akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala index bc3f370f5b..ebe29a8219 100644 --- a/akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala +++ b/akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala @@ -8,7 +8,7 @@ import annotation.tailrec import akka.actor.{ Props, Actor } import akka.util.duration._ import akka.dispatch.Await -import akka.patterns.ask +import akka.pattern.ask case class FibonacciNumber(nbr: Int) diff --git a/akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala b/akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala index b3ff8d2e05..89d95b3252 100644 --- a/akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala @@ -120,7 +120,7 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender { import akka.testkit.TestActorRef import akka.util.duration._ import akka.dispatch.Await - import akka.patterns.ask + import akka.pattern.ask val actorRef = TestActorRef(new MyActor) // hypothetical message stimulating a '42' answer @@ -204,7 +204,7 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender { "demonstrate probe reply" in { import akka.testkit.TestProbe import akka.util.duration._ - import akka.patterns.ask + import akka.pattern.ask //#test-probe-reply val probe = TestProbe() val future = probe.ref ? "hello" diff --git a/akka-docs/scala/code/akka/docs/transactor/TransactorDocSpec.scala b/akka-docs/scala/code/akka/docs/transactor/TransactorDocSpec.scala index 2cde9c9890..ed94589499 100644 --- a/akka-docs/scala/code/akka/docs/transactor/TransactorDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/transactor/TransactorDocSpec.scala @@ -141,7 +141,7 @@ class TransactorDocSpec extends AkkaSpec { import akka.dispatch.Await import akka.util.duration._ import akka.util.Timeout - import akka.patterns.ask + import akka.pattern.ask val system = ActorSystem("app") diff --git a/akka-remote/src/main/scala/akka/remote/Gossiper.scala b/akka-remote/src/main/scala/akka/remote/Gossiper.scala index 3ff54b5efe..788e86bb1c 100644 --- a/akka-remote/src/main/scala/akka/remote/Gossiper.scala +++ b/akka-remote/src/main/scala/akka/remote/Gossiper.scala @@ -22,7 +22,7 @@ import scala.annotation.tailrec import com.google.protobuf.ByteString import java.util.concurrent.TimeoutException import akka.dispatch.Await -import akka.patterns.ask +import akka.pattern.ask /** * Interface for node membership change listener. diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala index 58c52d8971..36ba02a980 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala @@ -5,7 +5,7 @@ import akka.routing._ import akka.actor.{ Actor, Props } import akka.testkit._ import akka.dispatch.Await -import akka.patterns.ask +import akka.pattern.ask object DirectRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { override def NrOfNodes = 2 diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala index cc8f62552c..cfdb6ae2dd 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala @@ -6,7 +6,7 @@ import akka.routing._ import akka.testkit._ import akka.util.duration._ import akka.dispatch.Await -import akka.patterns.ask +import akka.pattern.ask object NewRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { override def NrOfNodes = 2 diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala index d466c014b3..2c81d9d03d 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala @@ -5,7 +5,7 @@ import akka.remote._ import akka.routing._ import akka.testkit.DefaultTimeout import akka.dispatch.Await -import akka.patterns.ask +import akka.pattern.ask object RandomRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { override def NrOfNodes = 4 diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala index 22eb8cd8c3..8381e14755 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala @@ -5,7 +5,7 @@ import akka.remote._ import akka.routing._ import akka.testkit.DefaultTimeout import akka.dispatch.Await -import akka.patterns.ask +import akka.pattern.ask object RoundRobinRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { override def NrOfNodes = 4 diff --git a/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala index 17068cc351..4f1fb545c0 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala @@ -7,7 +7,7 @@ import akka.testkit._ import akka.actor._ import com.typesafe.config._ import akka.dispatch.Await -import akka.patterns.ask +import akka.pattern.ask object RemoteCommunicationSpec { class Echo extends Actor { diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala index 01ce145afa..eb9d12b56a 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala @@ -13,7 +13,7 @@ import java.util.concurrent.atomic.AtomicLong import akka.event.EventStream import scala.collection.immutable.Stack import akka.dispatch._ -import akka.patterns.ask +import akka.pattern.ask /** * This special ActorRef is exclusively for use during unit testing in a single-threaded environment. Therefore, it diff --git a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala index e47587fc9c..329d11fc39 100644 --- a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala @@ -17,7 +17,7 @@ import akka.actor.DeadLetter import java.util.concurrent.TimeoutException import akka.dispatch.{ Await, MessageDispatcher } import akka.dispatch.Dispatchers -import akka.patterns.ask +import akka.pattern.ask object TimingTest extends Tag("timing") diff --git a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala index b742165f1d..18359fcfda 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala @@ -10,7 +10,7 @@ import akka.event.Logging.Warning import akka.dispatch.{ Future, Promise, Await } import akka.util.duration._ import akka.actor.ActorSystem -import akka.patterns.ask +import akka.pattern.ask /** * Test whether TestActorRef behaves as an ActorRef should, besides its own spec. diff --git a/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala index 6a4a9db420..516bba7322 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala @@ -6,7 +6,7 @@ import org.scalatest.{ BeforeAndAfterEach, WordSpec } import akka.actor._ import akka.util.duration._ import akka.dispatch.{ Await, Future } -import akka.patterns.ask +import akka.pattern.ask @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class TestProbeSpec extends AkkaSpec with DefaultTimeout { diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java b/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java index d282a638fc..3ae9046c8b 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java @@ -18,7 +18,7 @@ import akka.actor.UntypedActor; import akka.actor.UntypedActorFactory; import akka.dispatch.Await; import akka.dispatch.Future; -import static akka.Patterns.ask; +import static akka.pattern.Patterns.ask; import akka.testkit.AkkaSpec; import akka.testkit.EventFilter; import akka.testkit.ErrorFilter; diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java b/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java index 2a6e30ea81..8898ec096f 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java @@ -18,7 +18,7 @@ import akka.actor.UntypedActor; import akka.actor.UntypedActorFactory; import akka.dispatch.Await; import akka.dispatch.Future; -import static akka.Patterns.ask; +import static akka.pattern.Patterns.ask; import akka.testkit.AkkaSpec; import akka.testkit.EventFilter; import akka.testkit.ErrorFilter; diff --git a/akka-transactor/src/test/scala/akka/transactor/CoordinatedIncrementSpec.scala b/akka-transactor/src/test/scala/akka/transactor/CoordinatedIncrementSpec.scala index 2c2e4376e9..b5319b2dc2 100644 --- a/akka-transactor/src/test/scala/akka/transactor/CoordinatedIncrementSpec.scala +++ b/akka-transactor/src/test/scala/akka/transactor/CoordinatedIncrementSpec.scala @@ -12,7 +12,7 @@ import akka.util.duration._ import akka.util.Timeout import akka.testkit._ import scala.concurrent.stm._ -import akka.patterns.ask +import akka.pattern.ask object CoordinatedIncrement { case class Increment(friends: Seq[ActorRef]) diff --git a/akka-transactor/src/test/scala/akka/transactor/FickleFriendsSpec.scala b/akka-transactor/src/test/scala/akka/transactor/FickleFriendsSpec.scala index a8e4e1ad0e..7184c7d520 100644 --- a/akka-transactor/src/test/scala/akka/transactor/FickleFriendsSpec.scala +++ b/akka-transactor/src/test/scala/akka/transactor/FickleFriendsSpec.scala @@ -15,7 +15,7 @@ import akka.testkit.TestEvent.Mute import scala.concurrent.stm._ import scala.util.Random.{ nextInt ⇒ random } import java.util.concurrent.CountDownLatch -import akka.patterns.ask +import akka.pattern.ask object FickleFriends { case class FriendlyIncrement(friends: Seq[ActorRef], timeout: Timeout, latch: CountDownLatch) diff --git a/akka-transactor/src/test/scala/akka/transactor/TransactorSpec.scala b/akka-transactor/src/test/scala/akka/transactor/TransactorSpec.scala index 2d5797c97a..bba7c20612 100644 --- a/akka-transactor/src/test/scala/akka/transactor/TransactorSpec.scala +++ b/akka-transactor/src/test/scala/akka/transactor/TransactorSpec.scala @@ -10,7 +10,7 @@ import akka.util.duration._ import akka.util.Timeout import akka.testkit._ import scala.concurrent.stm._ -import akka.patterns.ask +import akka.pattern.ask object TransactorIncrement { case class Increment(friends: Seq[ActorRef], latch: TestLatch) From 00ec3f89dcb51191b23093570ca56324c167dd14 Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 18 Jan 2012 11:52:35 +0100 Subject: [PATCH 021/152] move ask machinery from ActorRefProvider to pattern.AskSupport - now everything is assembled in one spot - also moved PromiseActorRef and AskTimeoutException from akka.actor - plus a little boy-scouting --- .../scala/akka/actor/ActorTimeoutSpec.scala | 2 +- .../scala/akka/actor/ForwardActorSpec.scala | 2 +- .../src/main/scala/akka/actor/ActorRef.scala | 38 ----- .../scala/akka/actor/ActorRefProvider.scala | 55 ++++--- .../main/scala/akka/pattern/AskSupport.scala | 142 ++++++++++++++++++ .../scala/akka/pattern/AskableActorRef.scala | 84 ----------- .../main/scala/akka/pattern/Patterns.scala | 32 ++++ .../src/main/scala/akka/pattern/package.scala | 20 +-- .../src/main/scala/akka/routing/Routing.scala | 3 +- .../akka/remote/RemoteActorRefProvider.scala | 7 +- .../akka/remote/RemoteCommunicationSpec.scala | 4 +- 11 files changed, 226 insertions(+), 163 deletions(-) create mode 100644 akka-actor/src/main/scala/akka/pattern/AskSupport.scala delete mode 100644 akka-actor/src/main/scala/akka/pattern/AskableActorRef.scala diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala index e5b2303140..de1dd88923 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala @@ -10,7 +10,7 @@ import akka.testkit.DefaultTimeout import java.util.concurrent.TimeoutException import akka.dispatch.Await import akka.util.Timeout -import akka.pattern.ask +import akka.pattern.{ ask, AskTimeoutException } @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class ActorTimeoutSpec extends AkkaSpec with BeforeAndAfterAll with DefaultTimeout { diff --git a/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala index 95ecfe1280..cc5eb4c3a9 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala @@ -47,7 +47,7 @@ class ForwardActorSpec extends AkkaSpec { "forward actor reference when invoking forward on ask" in { val chain = createForwardingChain(system) - chain.ask(ExpectedMessage)(5000) onSuccess { case ExpectedMessage ⇒ testActor ! ExpectedMessage } + chain.ask(ExpectedMessage)(5 seconds) onSuccess { case ExpectedMessage ⇒ testActor ! ExpectedMessage } expectMsg(5 seconds, ExpectedMessage) } } diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 9ebae64e62..a053608a1b 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -462,41 +462,3 @@ class VirtualPathContainer(val path: ActorPath, override val getParent: Internal } } } - -/** - * This is what is used to complete a Future that is returned from an ask/? call, - * when it times out. - */ -class AskTimeoutException(message: String, cause: Throwable) extends TimeoutException { - def this(message: String) = this(message, null: Throwable) -} - -/** - * Akka private optimized representation of the temporary actor spawned to - * receive the reply to an "ask" operation. - */ -private[akka] final class PromiseActorRef( - val path: ActorPath, - override val getParent: InternalActorRef, - val result: Promise[Any], - val deathWatch: DeathWatch) extends MinimalActorRef { - - final val running = new AtomicBoolean(true) - - override def !(message: Any)(implicit sender: ActorRef = null): Unit = if (running.get) message match { - case Status.Success(r) ⇒ result.success(r) - case Status.Failure(f) ⇒ result.failure(f) - case other ⇒ result.success(other) - } - - override def sendSystemMessage(message: SystemMessage): Unit = message match { - case _: Terminate ⇒ stop() - case _ ⇒ - } - - override def isTerminated = result.isCompleted - - override def stop(): Unit = if (running.getAndSet(false)) { - deathWatch.publish(Terminated(this)) - } -} diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 4408f7562e..50478a248f 100755 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -70,6 +70,26 @@ trait ActorRefProvider { def scheduler: Scheduler + /** + * Generates and returns a unique actor path below “/temp”. + */ + def tempPath(): ActorPath + + /** + * Returns the actor reference representing the “/temp” path. + */ + def tempContainer: InternalActorRef + + /** + * Registers an actorRef at a path returned by tempPath(); do NOT pass in any other path. + */ + def registerTempActor(actorRef: InternalActorRef, path: ActorPath) + + /** + * Unregister a temporary actor from the “/temp” path (i.e. obtained from tempPath()); do NOT pass in any other path. + */ + def unregisterTempActor(path: ActorPath) + /** * Actor factory with create-only semantics: will create an actor as * described by props with the given supervisor and path (may be different @@ -100,12 +120,6 @@ trait ActorRefProvider { */ def actorFor(ref: InternalActorRef, p: Iterable[String]): InternalActorRef - /** - * Create AskActorRef and register it properly so it can be serialized/deserialized; - * caller needs to send the message. - */ - def ask(within: Timeout): Option[PromiseActorRef] - /** * This Future is completed upon termination of this ActorRefProvider, which * is usually initiated by stopping the guardian via ActorSystem.stop(). @@ -441,6 +455,16 @@ class LocalActorRefProvider( lazy val tempContainer = new VirtualPathContainer(tempNode, rootGuardian, log) + def registerTempActor(actorRef: InternalActorRef, path: ActorPath): Unit = { + assert(path.parent eq tempNode, "cannot registerTempActor() with anything not obtained from tempPath()") + tempContainer.addChild(path.name, actorRef) + } + + def unregisterTempActor(path: ActorPath): Unit = { + assert(path.parent eq tempNode, "cannot unregisterTempActor() with anything not obtained from tempPath()") + tempContainer.removeChild(path.name) + } + val deathWatch = new LocalDeathWatch(1024) //TODO make configrable def init(_system: ActorSystemImpl) { @@ -493,25 +517,6 @@ class LocalActorRefProvider( new RoutedActorRef(system, props.withRouter(router.adaptFromDeploy(depl)), supervisor, path) } } - - def ask(within: Timeout): Option[PromiseActorRef] = { - (if (within == null) settings.ActorTimeout else within) match { - case t if t.duration.length <= 0 ⇒ None - case t ⇒ - val path = tempPath() - val name = path.name - val result = Promise[Any]()(dispatcher) - val a = new PromiseActorRef(path, tempContainer, result, deathWatch) - tempContainer.addChild(name, a) - val f = dispatcher.prerequisites.scheduler.scheduleOnce(t.duration) { result.failure(new AskTimeoutException("Timed out")) } - result onComplete { _ ⇒ - try { a.stop(); f.cancel() } - finally { tempContainer.removeChild(name) } - } - - Some(a) - } - } } class LocalDeathWatch(val mapSize: Int) extends DeathWatch with ActorClassification { diff --git a/akka-actor/src/main/scala/akka/pattern/AskSupport.scala b/akka-actor/src/main/scala/akka/pattern/AskSupport.scala new file mode 100644 index 0000000000..bc79877fc3 --- /dev/null +++ b/akka-actor/src/main/scala/akka/pattern/AskSupport.scala @@ -0,0 +1,142 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka.pattern + +import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.TimeoutException +import akka.actor.{ Terminated, Status, MinimalActorRef, InternalActorRef, ActorRef, ActorPath } +import akka.dispatch.{ Promise, Terminate, SystemMessage, Future } +import akka.event.DeathWatch +import akka.actor.ActorRefProvider +import akka.util.Timeout + +/** + * This is what is used to complete a Future that is returned from an ask/? call, + * when it times out. + */ +class AskTimeoutException(message: String, cause: Throwable) extends TimeoutException { + def this(message: String) = this(message, null: Throwable) +} + +object AskSupport { + + final class AskableActorRef(val actorRef: ActorRef) { + + /** + * 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 + * given timeout has expired; this is independent from any timeout applied + * while awaiting a result for this future (i.e. in + * `Await.result(..., timeout)`). + * + * Warning: + * When using future callbacks, inside actors you need to carefully avoid closing over + * the containing actor’s object, i.e. do not call methods or access mutable state + * on the enclosing actor from within the callback. This would break the actor + * encapsulation and may introduce synchronization bugs and race conditions because + * the callback will be scheduled concurrently to the enclosing actor. Unfortunately + * there is not yet a way to detect these illegal accesses at compile time. + * + * Recommended usage: + * + * {{{ + * val f = worker.ask(request)(timeout) + * flow { + * EnrichedRequest(request, f()) + * } pipeTo nextActor + * }}} + * + * [see the [[akka.dispatch.Future]] companion object for a description of `flow`] + */ + def ask(message: AnyRef)(implicit timeout: Timeout): Future[Any] = akka.pattern.ask(actorRef, message)(timeout) + + /** + * 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 + * given timeout has expired; this is independent from any timeout applied + * while awaiting a result for this future (i.e. in + * `Await.result(..., timeout)`). + * + * Warning: + * When using future callbacks, inside actors you need to carefully avoid closing over + * the containing actor’s object, i.e. do not call methods or access mutable state + * on the enclosing actor from within the callback. This would break the actor + * encapsulation and may introduce synchronization bugs and race conditions because + * the callback will be scheduled concurrently to the enclosing actor. Unfortunately + * there is not yet a way to detect these illegal accesses at compile time. + * + * Recommended usage: + * + * {{{ + * val f = worker ? request + * flow { + * EnrichedRequest(request, f()) + * } pipeTo nextActor + * }}} + * + * [see the [[akka.dispatch.Future]] companion object for a description of `flow`] + */ + def ?(message: Any)(implicit timeout: Timeout): Future[Any] = akka.pattern.ask(actorRef, message) + + /* + * FIXME: I think this should be removed, since it introduces an “ambiguity” + * when sending Tuple2, which the compiler resolves unexpectedly to this + * method; also overloading is bad, isn’t it? - RK (ticket #1653) + */ + /** + * Sends a message asynchronously, returning a future which may eventually hold the reply. + * The implicit parameter with the default value is just there to disambiguate it from the version that takes the + * implicit timeout + */ + def ?(message: Any, timeout: Timeout)(implicit ignore: Int = 0): Future[Any] = ?(message)(timeout) + + } + + /** + * Akka private optimized representation of the temporary actor spawned to + * receive the reply to an "ask" operation. + */ + private[akka] final class PromiseActorRef( + val path: ActorPath, + override val getParent: InternalActorRef, + val result: Promise[Any], + val deathWatch: DeathWatch) extends MinimalActorRef { + + final val running = new AtomicBoolean(true) + + override def !(message: Any)(implicit sender: ActorRef = null): Unit = if (running.get) message match { + case Status.Success(r) ⇒ result.success(r) + case Status.Failure(f) ⇒ result.failure(f) + case other ⇒ result.success(other) + } + + override def sendSystemMessage(message: SystemMessage): Unit = message match { + case _: Terminate ⇒ stop() + case _ ⇒ + } + + override def isTerminated = result.isCompleted + + override def stop(): Unit = if (running.getAndSet(false)) { + deathWatch.publish(Terminated(this)) + } + } + + def createAsker(provider: ActorRefProvider, timeout: Timeout): PromiseActorRef = { + val path = provider.tempPath() + val result = Promise[Any]()(provider.dispatcher) + val a = new PromiseActorRef(path, provider.tempContainer, result, provider.deathWatch) + provider.registerTempActor(a, path) + val f = provider.scheduler.scheduleOnce(timeout.duration) { result.failure(new AskTimeoutException("Timed out")) } + result onComplete { _ ⇒ + try { a.stop(); f.cancel() } + finally { provider.unregisterTempActor(path) } + } + a + } +} diff --git a/akka-actor/src/main/scala/akka/pattern/AskableActorRef.scala b/akka-actor/src/main/scala/akka/pattern/AskableActorRef.scala deleted file mode 100644 index faa4c4d5c0..0000000000 --- a/akka-actor/src/main/scala/akka/pattern/AskableActorRef.scala +++ /dev/null @@ -1,84 +0,0 @@ -/** - * Copyright (C) 2009-2011 Typesafe Inc. - */ -package akka.pattern - -import akka.actor.{ ActorRef, InternalActorRef, AskTimeoutException } -import akka.dispatch.{ Future, Promise } -import akka.util.Timeout - -final class AskableActorRef(val actorRef: ActorRef) { - - /** - * 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 - * given timeout has expired; this is independent from any timeout applied - * while awaiting a result for this future (i.e. in - * `Await.result(..., timeout)`). - * - * Warning: - * When using future callbacks, inside actors you need to carefully avoid closing over - * the containing actor’s object, i.e. do not call methods or access mutable state - * on the enclosing actor from within the callback. This would break the actor - * encapsulation and may introduce synchronization bugs and race conditions because - * the callback will be scheduled concurrently to the enclosing actor. Unfortunately - * there is not yet a way to detect these illegal accesses at compile time. - * - * Recommended usage: - * - * {{{ - * val f = worker.ask(request)(timeout) - * flow { - * EnrichedRequest(request, f()) - * } pipeTo nextActor - * }}} - * - * [see the [[akka.dispatch.Future]] companion object for a description of `flow`] - */ - def ask(message: AnyRef)(implicit timeout: Timeout): Future[Any] = akka.pattern.ask(actorRef, message)(timeout) - - /** - * 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 - * given timeout has expired; this is independent from any timeout applied - * while awaiting a result for this future (i.e. in - * `Await.result(..., timeout)`). - * - * Warning: - * When using future callbacks, inside actors you need to carefully avoid closing over - * the containing actor’s object, i.e. do not call methods or access mutable state - * on the enclosing actor from within the callback. This would break the actor - * encapsulation and may introduce synchronization bugs and race conditions because - * the callback will be scheduled concurrently to the enclosing actor. Unfortunately - * there is not yet a way to detect these illegal accesses at compile time. - * - * Recommended usage: - * - * {{{ - * val f = worker ? request - * flow { - * EnrichedRequest(request, f()) - * } pipeTo nextActor - * }}} - * - * [see the [[akka.dispatch.Future]] companion object for a description of `flow`] - */ - def ?(message: Any)(implicit timeout: Timeout): Future[Any] = akka.pattern.ask(actorRef, message) - - /* - * FIXME: I think this should be removed, since it introduces an “ambiguity” - * when sending Tuple2, which the compiler resolves unexpectedly to this - * method; also overloading is bad, isn’t it? - RK (ticket #1653) - */ - /** - * Sends a message asynchronously, returning a future which may eventually hold the reply. - * The implicit parameter with the default value is just there to disambiguate it from the version that takes the - * implicit timeout - */ - def ?(message: Any, timeout: Timeout)(implicit ignore: Int = 0): Future[Any] = ?(message)(timeout) - -} diff --git a/akka-actor/src/main/scala/akka/pattern/Patterns.scala b/akka-actor/src/main/scala/akka/pattern/Patterns.scala index bc86877f40..9f65728517 100644 --- a/akka-actor/src/main/scala/akka/pattern/Patterns.scala +++ b/akka-actor/src/main/scala/akka/pattern/Patterns.scala @@ -9,6 +9,38 @@ object Patterns { import akka.pattern.{ ask ⇒ scalaAsk } import akka.util.Timeout + /** + * 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 + * given timeout has expired; this is independent from any timeout applied + * while awaiting a result for this future (i.e. in + * `Await.result(..., timeout)`). + * + * This variant will use the `akka.actor.timeout` from the configuration. + * + * Warning: + * When using future callbacks, inside actors you need to carefully avoid closing over + * the containing actor’s object, i.e. do not call methods or access mutable state + * on the enclosing actor from within the callback. This would break the actor + * encapsulation and may introduce synchronization bugs and race conditions because + * the callback will be scheduled concurrently to the enclosing actor. Unfortunately + * there is not yet a way to detect these illegal accesses at compile time. + * + * Recommended usage: + * + * {{{ + * final Future f = Patterns.ask(worker, request, timeout); + * f.onSuccess(new Procedure() { + * public void apply(Object o) { + * nextActor.tell(new EnrichedResult(request, o)); + * } + * }); + * }}} + */ + def ask(actor: ActorRef, message: Any): Future[AnyRef] = scalaAsk(actor, message).asInstanceOf[Future[AnyRef]] + /** * Sends a message asynchronously and returns a [[akka.dispatch.Future]] * holding the eventual reply message; this means that the target actor diff --git a/akka-actor/src/main/scala/akka/pattern/package.scala b/akka-actor/src/main/scala/akka/pattern/package.scala index bfd6384c21..5bc426a67d 100644 --- a/akka-actor/src/main/scala/akka/pattern/package.scala +++ b/akka-actor/src/main/scala/akka/pattern/package.scala @@ -5,7 +5,7 @@ package akka package object pattern { - import akka.actor.{ ActorRef, InternalActorRef, ActorRefWithProvider, AskTimeoutException } + import akka.actor.{ ActorRef, InternalActorRef, ActorRefWithProvider } import akka.dispatch.{ Future, Promise } import akka.util.Timeout @@ -24,7 +24,7 @@ package object pattern { * * All of the above use an implicit [[akka.actor.Timeout]]. */ - implicit def ask(actorRef: ActorRef): AskableActorRef = new AskableActorRef(actorRef) + implicit def ask(actorRef: ActorRef): AskSupport.AskableActorRef = new AskSupport.AskableActorRef(actorRef) /** * Sends a message asynchronously and returns a [[akka.dispatch.Future]] @@ -54,15 +54,17 @@ package object pattern { * * [see [[akka.dispatch.Future]] for a description of `flow`] */ - def ask(actorRef: ActorRef, message: Any)(implicit timeout: Timeout): Future[Any] = actorRef match { + def ask(actorRef: ActorRef, message: Any)(implicit timeout: Timeout = null): Future[Any] = actorRef match { case ref: ActorRefWithProvider ⇒ - ref.provider.ask(timeout) match { - case Some(ref) ⇒ - actorRef.tell(message, ref) - ref.result - case None ⇒ + val provider = ref.provider + (if (timeout == null) provider.settings.ActorTimeout else timeout) match { + case t if t.duration.length <= 0 ⇒ actorRef.tell(message) - Promise.failed(new AskTimeoutException("failed to create PromiseActorRef"))(ref.provider.dispatcher) + Promise.failed(new AskTimeoutException("failed to create PromiseActorRef"))(provider.dispatcher) + case t ⇒ + val a = AskSupport.createAsker(provider, t) + actorRef.tell(message, a) + a.result } case _ ⇒ throw new IllegalArgumentException("incompatible ActorRef " + actorRef) } diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index 11f8fee4af..c2f7faaae2 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -9,6 +9,7 @@ import scala.collection.JavaConversions._ import akka.util.{ Duration, Timeout } import akka.config.ConfigurationException import akka.dispatch.Promise +import akka.pattern.AskSupport /** * A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to @@ -404,7 +405,7 @@ trait ScatterGatherFirstCompletedLike { this: RouterConfig ⇒ { case (sender, message) ⇒ val provider: ActorRefProvider = context.asInstanceOf[ActorCell].systemImpl.provider - val asker = provider.ask(Timeout(within)).get + val asker = AskSupport.createAsker(provider, within) asker.result.pipeTo(sender) toAll(asker, ref.routees) } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 65417d9063..fe81dc431e 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -39,6 +39,11 @@ class RemoteActorRefProvider( def terminationFuture = local.terminationFuture def dispatcher = local.dispatcher + def registerTempActor(actorRef: InternalActorRef, path: ActorPath) = local.registerTempActor(actorRef, path) + def unregisterTempActor(path: ActorPath) = local.unregisterTempActor(path) + def tempPath() = local.tempPath() + def tempContainer = local.tempContainer + val deployer = new RemoteDeployer(settings) val remote = new Remote(settings, remoteSettings) @@ -138,8 +143,6 @@ class RemoteActorRefProvider( def actorFor(ref: InternalActorRef, path: Iterable[String]): InternalActorRef = local.actorFor(ref, path) - def ask(within: Timeout): Option[PromiseActorRef] = local.ask(within) - /** * Using (checking out) actor on a specific node. */ diff --git a/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala index 4f1fb545c0..d9d815d018 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala @@ -83,8 +83,8 @@ akka { "support ask" in { Await.result(here ? "ping", timeout.duration) match { - case ("pong", s: PromiseActorRef) ⇒ // good - case m ⇒ fail(m + " was not (pong, AskActorRef)") + case ("pong", s: akka.pattern.AskSupport.PromiseActorRef) ⇒ // good + case m ⇒ fail(m + " was not (pong, AskActorRef)") } } From 4aa0a1ff2b10bbf69de1687ab8ccb8c731a114d4 Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 18 Jan 2012 13:01:24 +0100 Subject: [PATCH 022/152] add akka.pattern.pipeTo, see #1582 - move code from akka.actor package to akka.pattern - add Java API in akka.pattern.Patterns.pipeTo() --- .../main/scala/akka/pattern/Patterns.scala | 18 ++++++++++ .../scala/akka/pattern/PipeToSupport.scala | 15 ++++++++ .../src/main/scala/akka/pattern/package.scala | 35 +++++++++++++++++++ 3 files changed, 68 insertions(+) create mode 100644 akka-actor/src/main/scala/akka/pattern/PipeToSupport.scala diff --git a/akka-actor/src/main/scala/akka/pattern/Patterns.scala b/akka-actor/src/main/scala/akka/pattern/Patterns.scala index 9f65728517..50e6bb909c 100644 --- a/akka-actor/src/main/scala/akka/pattern/Patterns.scala +++ b/akka-actor/src/main/scala/akka/pattern/Patterns.scala @@ -100,4 +100,22 @@ object Patterns { * }}} */ def ask(actor: ActorRef, message: Any, timeoutMillis: Long): Future[AnyRef] = scalaAsk(actor, message)(new Timeout(timeoutMillis)).asInstanceOf[Future[AnyRef]] + + /** + * Register an onComplete callback on this [[akka.dispatch.Future]] to send + * the result to the given actor reference. Returns the original Future to + * allow method chaining. + * + * Recommended usage example: + * + * {{{ + * val f = ask(worker, request)(timeout) + * flow { + * EnrichedRequest(request, f()) + * } pipeTo nextActor + * }}} + * + * [see [[akka.dispatch.Future]] for a description of `flow`] + */ + def pipeTo[T](future: Future[T], actorRef: ActorRef): Future[T] = akka.pattern.pipeTo(future, actorRef) } diff --git a/akka-actor/src/main/scala/akka/pattern/PipeToSupport.scala b/akka-actor/src/main/scala/akka/pattern/PipeToSupport.scala new file mode 100644 index 0000000000..6827946902 --- /dev/null +++ b/akka-actor/src/main/scala/akka/pattern/PipeToSupport.scala @@ -0,0 +1,15 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka.pattern + +import akka.actor.ActorRef +import akka.dispatch.Future + +object PipeToSupport { + + class PipeableFuture[T](val future: Future[T]) { + def pipeTo(actorRef: ActorRef): Future[T] = akka.pattern.pipeTo(future, actorRef) + } + +} \ No newline at end of file diff --git a/akka-actor/src/main/scala/akka/pattern/package.scala b/akka-actor/src/main/scala/akka/pattern/package.scala index 5bc426a67d..500beca807 100644 --- a/akka-actor/src/main/scala/akka/pattern/package.scala +++ b/akka-actor/src/main/scala/akka/pattern/package.scala @@ -69,4 +69,39 @@ package object pattern { case _ ⇒ throw new IllegalArgumentException("incompatible ActorRef " + actorRef) } + /** + * Import this implicit conversion to gain the `pipeTo` method on [[akka.dispatch.Future]]: + * + * {{{ + * import akka.pattern.pipeTo + * + * Future { doExpensiveCalc() } pipeTo nextActor + * }}} + */ + implicit def pipeTo[T](future: Future[T]): PipeToSupport.PipeableFuture[T] = new PipeToSupport.PipeableFuture(future) + + /** + * Register an onComplete callback on this [[akka.dispatch.Future]] to send + * the result to the given actor reference. Returns the original Future to + * allow method chaining. + * + * Recommended usage example: + * + * {{{ + * val f = ask(worker, request)(timeout) + * flow { + * EnrichedRequest(request, f()) + * } pipeTo nextActor + * }}} + * + * [see [[akka.dispatch.Future]] for a description of `flow`] + */ + def pipeTo[T](future: Future[T], actorRef: ActorRef): Future[T] = { + future onComplete { + case Right(r) ⇒ actorRef ! r + case Left(f) ⇒ actorRef ! akka.actor.Status.Failure(f) + } + future + } + } From 8b14cd683d0abdef0d3dae15341d0e58acd4625a Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 18 Jan 2012 13:26:11 +0100 Subject: [PATCH 023/152] scaladoc for patterns --- .../src/main/scala/akka/actor/Scheduler.scala | 4 +-- .../main/scala/akka/pattern/Patterns.scala | 3 ++ .../src/main/scala/akka/pattern/package.scala | 33 +++++++++++++++++++ 3 files changed, 38 insertions(+), 2 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/Scheduler.scala b/akka-actor/src/main/scala/akka/actor/Scheduler.scala index a7d4376114..8104199dec 100644 --- a/akka-actor/src/main/scala/akka/actor/Scheduler.scala +++ b/akka-actor/src/main/scala/akka/actor/Scheduler.scala @@ -27,8 +27,8 @@ trait Scheduler { /** * Schedules a message to be sent repeatedly with an initial delay and * frequency. E.g. if you would like a message to be sent immediately and - * thereafter every 500ms you would set delay = Duration.Zero and frequency - * = Duration(500, TimeUnit.MILLISECONDS) + * thereafter every 500ms you would set delay=Duration.Zero and + * frequency=Duration(500, TimeUnit.MILLISECONDS) * * Java & Scala API */ diff --git a/akka-actor/src/main/scala/akka/pattern/Patterns.scala b/akka-actor/src/main/scala/akka/pattern/Patterns.scala index 50e6bb909c..ae359ba795 100644 --- a/akka-actor/src/main/scala/akka/pattern/Patterns.scala +++ b/akka-actor/src/main/scala/akka/pattern/Patterns.scala @@ -10,6 +10,7 @@ object Patterns { import akka.util.Timeout /** + * Java API for `akka.pattern.ask`: * 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 @@ -42,6 +43,7 @@ object Patterns { def ask(actor: ActorRef, message: Any): Future[AnyRef] = scalaAsk(actor, message).asInstanceOf[Future[AnyRef]] /** + * Java API for `akka.pattern.ask`: * 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 @@ -72,6 +74,7 @@ object Patterns { def ask(actor: ActorRef, message: Any, timeout: Timeout): Future[AnyRef] = scalaAsk(actor, message)(timeout).asInstanceOf[Future[AnyRef]] /** + * Java API for `akka.pattern.ask`: * 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 diff --git a/akka-actor/src/main/scala/akka/pattern/package.scala b/akka-actor/src/main/scala/akka/pattern/package.scala index 500beca807..41cba41876 100644 --- a/akka-actor/src/main/scala/akka/pattern/package.scala +++ b/akka-actor/src/main/scala/akka/pattern/package.scala @@ -3,6 +3,39 @@ */ package akka +/** + * == Commonly Used Patterns With Akka == + * + * This package is used as a collection point for usage patterns which involve + * actors, futures, etc. but are loosely enough coupled to (multiple of) them + * to present them separately from the core implementation. Currently supported + * are: + * + *
    + *
  • ask: create a temporary one-off actor for receiving a reply to a + * message and complete a [[akka.dispatch.Future]] with it; returns said + * Future.
  • + *
  • pipeTo: feed eventually computed value of a future to an actor as + * a message.
  • + *
+ * + * In Scala the recommended usage is to import the pattern from the package + * object: + * {{{ + * import akka.pattern.ask + * + * ask(actor, message) // use it directly + * actor ask message // use it by implicit conversion + * }}} + * + * For Java the patterns are available as static methods of the [[akka.pattern.Patterns]] + * class: + * {{{ + * import static akka.pattern.Patterns.ask; + * + * ask(actor, message); + * }}} + */ package object pattern { import akka.actor.{ ActorRef, InternalActorRef, ActorRefWithProvider } From a5c55fd017c972cdeaab659c106871cab4917d19 Mon Sep 17 00:00:00 2001 From: Ivan Porto Carrero Date: Sat, 14 Jan 2012 03:16:39 +0100 Subject: [PATCH 024/152] Adds initial version of zeromq support for akka 2.0 --- akka-actor/src/main/resources/reference.conf | 6 + .../akka/zeromq/ConcurrentSocketActor.scala | 207 ++++++++++++++++++ .../src/main/scala/akka/zeromq/Context.scala | 20 ++ .../main/scala/akka/zeromq/Deserializer.scala | 15 ++ .../src/main/scala/akka/zeromq/Requests.scala | 104 +++++++++ .../main/scala/akka/zeromq/Responses.scala | 8 + .../main/scala/akka/zeromq/SocketType.scala | 14 ++ .../scala/akka/zeromq/ZeroMQExtension.scala | 65 ++++++ .../zeromq/ConcurrentSocketActorSpec.scala | 149 +++++++++++++ project/AkkaBuild.scala | 14 ++ 10 files changed, 602 insertions(+) create mode 100644 akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala create mode 100644 akka-zeromq/src/main/scala/akka/zeromq/Context.scala create mode 100644 akka-zeromq/src/main/scala/akka/zeromq/Deserializer.scala create mode 100644 akka-zeromq/src/main/scala/akka/zeromq/Requests.scala create mode 100644 akka-zeromq/src/main/scala/akka/zeromq/Responses.scala create mode 100644 akka-zeromq/src/main/scala/akka/zeromq/SocketType.scala create mode 100644 akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala create mode 100644 akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index 94efc34176..68b614ed28 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -146,6 +146,12 @@ akka { } } + zeromq-dispatcher { + # A zeromq socket needs to be pinned to the thread that created it. + # Changing this value results in weird errors and race conditions within zeromq + type = "PinnedDispatcher" + } + default-dispatcher { # Must be one of the following # Dispatcher, (BalancingDispatcher, only valid when all actors using it are of diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala new file mode 100644 index 0000000000..a1a340e282 --- /dev/null +++ b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala @@ -0,0 +1,207 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka.zeromq + +import org.zeromq.ZMQ.{ Socket, Poller } +import org.zeromq.{ ZMQ ⇒ JZMQ } +import akka.actor._ +import akka.dispatch.{ Promise, Dispatchers, Future } + +private[zeromq] sealed trait PollLifeCycle +private[zeromq] case object NoResults extends PollLifeCycle +private[zeromq] case object Results extends PollLifeCycle +private[zeromq] case object Closing extends PollLifeCycle + +private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Actor { + + private val noBytes = Array[Byte]() + private val socket: Socket = params.context.socket(params.socketType) + private val poller: Poller = params.context.poller + + override def receive: Receive = { + case Send(frames) ⇒ + sendFrames(frames) + pollAndReceiveFrames() + case ZMQMessage(frames) ⇒ + sendFrames(frames) + pollAndReceiveFrames() + case Connect(endpoint) ⇒ + socket.connect(endpoint) + notifyListener(Connecting) + pollAndReceiveFrames() + case Bind(endpoint) ⇒ + socket.bind(endpoint) + pollAndReceiveFrames() + case Subscribe(topic) ⇒ + socket.subscribe(topic.toArray) + pollAndReceiveFrames() + case Unsubscribe(topic) ⇒ + socket.unsubscribe(topic.toArray) + pollAndReceiveFrames() + case Linger(value) ⇒ + socket.setLinger(value) + case Linger ⇒ + sender ! socket.getLinger + case ReconnectIVL ⇒ + sender ! socket.getReconnectIVL + case ReconnectIVL(value) ⇒ + socket.setReconnectIVL(value) + case Backlog ⇒ + sender ! socket.getBacklog + case Backlog(value) ⇒ + socket.setBacklog(value) + case ReconnectIVLMax ⇒ + sender ! socket.getReconnectIVLMax + case ReconnectIVLMax(value) ⇒ + socket.setReconnectIVLMax(value) + case MaxMsgSize ⇒ + sender ! socket.getMaxMsgSize + case MaxMsgSize(value) ⇒ + socket.setMaxMsgSize(value) + case SndHWM ⇒ + sender ! socket.getSndHWM + case SndHWM(value) ⇒ + socket.setSndHWM(value) + case RcvHWM ⇒ + sender ! socket.getRcvHWM + case RcvHWM(value) ⇒ + socket.setRcvHWM(value) + case HWM(value) ⇒ + socket.setHWM(value) + case Swap ⇒ + sender ! socket.getSwap + case Swap(value) ⇒ + socket.setSwap(value) + case Affinity ⇒ + sender ! socket.getAffinity + case Affinity(value) ⇒ + socket.setAffinity(value) + case Identity ⇒ + sender ! socket.getIdentity + case Identity(value) ⇒ + socket.setIdentity(value) + case Rate ⇒ + sender ! socket.getRate + case Rate(value) ⇒ + socket.setRate(value) + case RecoveryInterval ⇒ + sender ! socket.getRecoveryInterval + case RecoveryInterval(value) ⇒ + socket.setRecoveryInterval(value) + case MulticastLoop ⇒ + sender ! socket.hasMulticastLoop + case MulticastLoop(value) ⇒ + socket.setMulticastLoop(value) + case MulticastHops ⇒ + sender ! socket.getMulticastHops + case MulticastHops(value) ⇒ + socket.setMulticastHops(value) + case ReceiveTimeOut ⇒ + sender ! socket.getReceiveTimeOut + case ReceiveTimeOut(value) ⇒ + socket.setReceiveTimeOut(value) + case SendTimeOut ⇒ + sender ! socket.getSendTimeOut + case SendTimeOut(value) ⇒ + socket.setSendTimeOut(value) + case SendBufferSize ⇒ + sender ! socket.getSendBufferSize + case SendBufferSize(value) ⇒ + socket.setSendBufferSize(value) + case ReceiveBufferSize ⇒ + sender ! socket.getReceiveBufferSize + case ReceiveBufferSize(value) ⇒ + socket.setReceiveBufferSize(value) + case ReceiveMore ⇒ + sender ! socket.hasReceiveMore + case FileDescriptor ⇒ + sender ! socket.getFD + case 'poll ⇒ { + currentPoll = None + pollAndReceiveFrames() + } + case 'receiveFrames ⇒ { + receiveFrames() match { + case Seq() ⇒ + case frames ⇒ notifyListener(params.deserializer(frames)) + } + self ! 'poll + } + } + + override def preStart { + poller.register(socket, Poller.POLLIN) + } + + override def postStop { + currentPoll foreach { _ complete Right(Closing) } + poller.unregister(socket) + socket.close + notifyListener(Closed) + } + + private def sendFrames(frames: Seq[Frame]) { + def sendBytes(bytes: Seq[Byte], flags: Int) { + socket.send(bytes.toArray, flags) + } + val iter = frames.iterator + while (iter.hasNext) { + val payload = iter.next.payload + val flags = if (iter.hasNext) JZMQ.SNDMORE else 0 + sendBytes(payload, flags) + } + } + + private var currentPoll: Option[Promise[PollLifeCycle]] = None + private def pollAndReceiveFrames() { + currentPoll = currentPoll orElse Some(newEventLoop) + } + + private def newEventLoop: Promise[PollLifeCycle] = { + implicit val executor = context.system.dispatchers.defaultGlobalDispatcher + (Future { + if (poller.poll(params.pollTimeoutDuration.toMillis) > 0 && poller.pollin(0)) Results else NoResults + }).asInstanceOf[Promise[PollLifeCycle]] onSuccess { + case Results ⇒ if (!self.isTerminated) self ! 'receiveFrames + case NoResults ⇒ if (!self.isTerminated) self ! 'poll + case _ ⇒ currentPoll = None + } onFailure { + case ex ⇒ { + if (context.system != null) { + context.system.log.error(ex, "There was an error receiving messages on the zeromq socket") + } + if (!self.isTerminated) self ! 'poll + } + } + } + + private def receiveFrames(): Seq[Frame] = { + + @inline def receiveBytes(): Array[Byte] = socket.recv(0) match { + case null ⇒ noBytes + case bytes: Array[Byte] if bytes.length > 0 ⇒ bytes + case _ ⇒ noBytes + } + + receiveBytes() match { + case `noBytes` ⇒ Vector.empty + case someBytes ⇒ + var frames = Vector(Frame(someBytes)) + while (socket.hasReceiveMore) receiveBytes() match { + case `noBytes` ⇒ + case someBytes ⇒ frames :+= Frame(someBytes) + } + frames + } + } + + private def notifyListener(message: Any) { + params.listener.foreach { listener ⇒ + if (listener.isTerminated) + context stop self + else + listener ! message + } + } +} diff --git a/akka-zeromq/src/main/scala/akka/zeromq/Context.scala b/akka-zeromq/src/main/scala/akka/zeromq/Context.scala new file mode 100644 index 0000000000..073270436f --- /dev/null +++ b/akka-zeromq/src/main/scala/akka/zeromq/Context.scala @@ -0,0 +1,20 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka.zeromq + +import org.zeromq.{ ZMQ ⇒ JZMQ } +import akka.zeromq.SocketType._ + +class Context(numIoThreads: Int) { + private var context = JZMQ.context(numIoThreads) + def socket(socketType: SocketType) = { + context.socket(socketType.id) + } + def poller = { + context.poller + } + def term = { + context.term + } +} diff --git a/akka-zeromq/src/main/scala/akka/zeromq/Deserializer.scala b/akka-zeromq/src/main/scala/akka/zeromq/Deserializer.scala new file mode 100644 index 0000000000..6430a5a9c6 --- /dev/null +++ b/akka-zeromq/src/main/scala/akka/zeromq/Deserializer.scala @@ -0,0 +1,15 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka.zeromq + +case class Frame(payload: Seq[Byte]) +object Frame { def apply(s: String): Frame = Frame(s.getBytes) } + +trait Deserializer { + def apply(frames: Seq[Frame]): Any +} + +class ZMQMessageDeserializer extends Deserializer { + def apply(frames: Seq[Frame]) = ZMQMessage(frames) +} diff --git a/akka-zeromq/src/main/scala/akka/zeromq/Requests.scala b/akka-zeromq/src/main/scala/akka/zeromq/Requests.scala new file mode 100644 index 0000000000..a1ce8f6c5e --- /dev/null +++ b/akka-zeromq/src/main/scala/akka/zeromq/Requests.scala @@ -0,0 +1,104 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka.zeromq + +import com.google.protobuf.Message + +sealed trait Request +sealed trait SocketOption extends Request +sealed trait SocketOptionQuery extends Request + +case class Connect(endpoint: String) extends Request +case class Bind(endpoint: String) extends Request +private[zeromq] case object Close extends Request + +case class Subscribe(payload: Seq[Byte]) extends Request +object Subscribe { + def apply(topic: String): Subscribe = { + Subscribe(topic.getBytes) + } +} + +case class Unsubscribe(payload: Seq[Byte]) extends Request +object Unsubscribe { + def apply(topic: String): Unsubscribe = { + Unsubscribe(topic.getBytes) + } +} + +case class Send(frames: Seq[Frame]) extends Request + +case class ZMQMessage(frames: Seq[Frame]) { + def firstFrameAsString = { + new String(frames.head.payload.toArray) + } +} +object ZMQMessage { + def apply(bytes: Array[Byte]): ZMQMessage = { + ZMQMessage(Seq(Frame(bytes))) + } + def apply(message: Message): ZMQMessage = { + ZMQMessage(message.toByteArray) + } +} + +case class Linger(value: Long) extends SocketOption +object Linger extends SocketOptionQuery + +case class ReconnectIVL(value: Long) extends SocketOption +object ReconnectIVL extends SocketOptionQuery + +case class Backlog(value: Long) extends SocketOption +object Backlog extends SocketOptionQuery + +case class ReconnectIVLMax(value: Long) extends SocketOption +object ReconnectIVLMax extends SocketOptionQuery + +case class MaxMsgSize(value: Long) extends SocketOption +object MaxMsgSize extends SocketOptionQuery + +case class SndHWM(value: Long) extends SocketOption +object SndHWM extends SocketOptionQuery + +case class RcvHWM(value: Long) extends SocketOption +object RcvHWM extends SocketOptionQuery + +case class HWM(value: Long) extends SocketOption +/* object HWM extends SocketOptionQuery */ + +case class Swap(value: Long) extends SocketOption +object Swap extends SocketOptionQuery + +case class Affinity(value: Long) extends SocketOption +object Affinity extends SocketOptionQuery + +case class Identity(value: Array[Byte]) extends SocketOption +object Identity extends SocketOptionQuery + +case class Rate(value: Long) extends SocketOption +object Rate extends SocketOptionQuery + +case class RecoveryInterval(value: Long) extends SocketOption +object RecoveryInterval extends SocketOptionQuery + +case class MulticastLoop(value: Boolean) extends SocketOption +object MulticastLoop extends SocketOptionQuery + +case class MulticastHops(value: Long) extends SocketOption +object MulticastHops extends SocketOptionQuery + +case class ReceiveTimeOut(value: Long) extends SocketOption +object ReceiveTimeOut extends SocketOptionQuery + +case class SendTimeOut(value: Long) extends SocketOption +object SendTimeOut extends SocketOptionQuery + +case class SendBufferSize(value: Long) extends SocketOption +object SendBufferSize extends SocketOptionQuery + +case class ReceiveBufferSize(value: Long) extends SocketOption +object ReceiveBufferSize extends SocketOptionQuery + +object ReceiveMore extends SocketOptionQuery +object FileDescriptor extends SocketOptionQuery diff --git a/akka-zeromq/src/main/scala/akka/zeromq/Responses.scala b/akka-zeromq/src/main/scala/akka/zeromq/Responses.scala new file mode 100644 index 0000000000..43200a959c --- /dev/null +++ b/akka-zeromq/src/main/scala/akka/zeromq/Responses.scala @@ -0,0 +1,8 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka.zeromq + +sealed trait Response +case object Connecting extends Response +case object Closed extends Response diff --git a/akka-zeromq/src/main/scala/akka/zeromq/SocketType.scala b/akka-zeromq/src/main/scala/akka/zeromq/SocketType.scala new file mode 100644 index 0000000000..aba0c1608a --- /dev/null +++ b/akka-zeromq/src/main/scala/akka/zeromq/SocketType.scala @@ -0,0 +1,14 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka.zeromq + +import org.zeromq.{ ZMQ ⇒ JZMQ } + +object SocketType extends Enumeration { + type SocketType = Value + val Pub = Value(JZMQ.PUB) + val Sub = Value(JZMQ.SUB) + val Dealer = Value(JZMQ.DEALER) + val Router = Value(JZMQ.ROUTER) +} diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala new file mode 100644 index 0000000000..2515647050 --- /dev/null +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala @@ -0,0 +1,65 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka.zeromq + +import akka.util.Duration +import akka.util.duration._ +import akka.zeromq.SocketType._ +import org.zeromq.{ ZMQ ⇒ JZMQ } +import akka.actor._ +import akka.dispatch.Await + +case class SocketParameters( + socketType: SocketType, + context: Context, + listener: Option[ActorRef] = None, + deserializer: Deserializer = new ZMQMessageDeserializer, + pollTimeoutDuration: Duration = 100 millis) + +case class ZeroMQVersion(major: Int, minor: Int, patch: Int) { + override def toString = "%d.%d.%d".format(major, minor, patch) +} + +object ZeroMQExtension extends ExtensionId[ZeroMQExtension] with ExtensionIdProvider { + def lookup() = this + def createExtension(system: ActorSystemImpl) = new ZeroMQExtension(system) +} +class ZeroMQExtension(system: ActorSystem) extends Extension { + + def version = { + ZeroMQVersion(JZMQ.getMajorVersion, JZMQ.getMinorVersion, JZMQ.getPatchVersion) + } + + lazy val DefaultContext = newContext() + + def newContext(numIoThreads: Int = 1) = { + verifyZeroMQVersion + new Context(numIoThreads) + } + + def newSocket(socketType: SocketType, + listener: Option[ActorRef] = None, + context: Context = DefaultContext, // For most applications you want to use the default context + deserializer: Deserializer = new ZMQMessageDeserializer, + pollTimeoutDuration: Duration = 100 millis) = { + verifyZeroMQVersion + val params = SocketParameters(socketType, context, listener, deserializer, pollTimeoutDuration) + implicit val timeout = system.settings.ActorTimeout + val req = (zeromq ? Props(new ConcurrentSocketActor(params)).withDispatcher("zmqdispatcher")).mapTo[ActorRef] + Await.result(req, timeout.duration) + } + + val zeromq: ActorRef = { + verifyZeroMQVersion + system.asInstanceOf[ActorSystemImpl].systemActorOf(Props(new Actor { + protected def receive = { case p: Props ⇒ sender ! context.actorOf(p) } + }), "zeromq") + } + + private def verifyZeroMQVersion = { + require( + JZMQ.getFullVersion > JZMQ.makeVersion(2, 1, 0), + "Unsupported ZeroMQ version: %s".format(JZMQ.getVersionString)) + } +} diff --git a/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala b/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala new file mode 100644 index 0000000000..3b7af45196 --- /dev/null +++ b/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala @@ -0,0 +1,149 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka.zeromq + +import org.scalatest.matchers.MustMatchers +import akka.testkit.{ TestProbe, DefaultTimeout, AkkaSpec } +import akka.actor.{ Actor, Props, ActorRef } +import akka.util.Timeout +import akka.util.duration._ +import java.net.{ SocketException, ConnectException, Socket } +import util.Random + +object ConcurrentSocketActorSpec { + val config = """ +akka { + extensions = ["akka.zeromq.ZeroMQExtension$"] + actor { + zmqdispatcher { + type = "PinnedDispatcher" + } + } +} +""" +} + +class ConcurrentSocketActorSpec extends AkkaSpec(ConcurrentSocketActorSpec.config) with MustMatchers with DefaultTimeout { + + val endpoint = "tcp://127.0.0.1:%s" format FreePort.randomFreePort() + + def zmq = system.extension(ZeroMQExtension) + + "ConcurrentSocketActor" should { + "support pub-sub connections" in { + checkZeroMQInstallation + val (publisherProbe, subscriberProbe) = (TestProbe(), TestProbe()) + val context = zmq.newContext() + val publisher = newPublisher(context, publisherProbe.ref) + val subscriber = newSubscriber(context, subscriberProbe.ref) + val msgGenerator = newMessageGenerator(publisher) + + try { + subscriberProbe.expectMsg(Connecting) + val msgNumbers = subscriberProbe.receiveWhile(2 seconds) { + case msg: ZMQMessage ⇒ { + println("RECV: " + msg.firstFrameAsString) + msg + } + }.map(_.firstFrameAsString.toInt) + msgNumbers.length must be > 0 + msgNumbers must equal(for (i ← msgNumbers.head to msgNumbers.last) yield i) + } finally { + system stop msgGenerator + within(2 seconds) { awaitCond(msgGenerator.isTerminated) } + system stop subscriber + system stop publisher + subscriberProbe.receiveWhile(1 seconds) { + case msg ⇒ msg + }.last must equal(Closed) + context.term + } + } + "support zero-length message frames" in { + checkZeroMQInstallation + val publisherProbe = TestProbe() + val context = zmq.newContext() + val publisher = newPublisher(context, publisherProbe.ref) + + try { + publisher ! ZMQMessage(Seq[Frame]()) + } finally { + system stop publisher + publisherProbe.within(5 seconds) { + publisherProbe.expectMsg(Closed) + } + context.term + } + } + def newPublisher(context: Context, listener: ActorRef) = { + val publisher = zmq.newSocket(SocketType.Pub, context = context, listener = Some(listener)) + publisher ! Bind(endpoint) + publisher + } + def newSubscriber(context: Context, listener: ActorRef) = { + val subscriber = zmq.newSocket(SocketType.Sub, context = context, listener = Some(listener)) + subscriber ! Connect(endpoint) + subscriber ! Subscribe(Seq()) + subscriber + } + def newMessageGenerator(actorRef: ActorRef) = { + system.actorOf(Props(new MessageGeneratorActor(actorRef)).withTimeout(Timeout(10 millis))) + } + def checkZeroMQInstallation = try { + zmq.version match { + case ZeroMQVersion(2, 1, _) ⇒ Unit + case version ⇒ invalidZeroMQVersion(version) + } + } catch { + case e: LinkageError ⇒ zeroMQNotInstalled + } + def invalidZeroMQVersion(version: ZeroMQVersion) { + info("WARNING: The tests are not run because invalid ZeroMQ version: %s. Version >= 2.1.x required.".format(version)) + pending + } + def zeroMQNotInstalled { + info("WARNING: The tests are not run because ZeroMQ is not installed. Version >= 2.1.x required.") + pending + } + } + class MessageGeneratorActor(actorRef: ActorRef) extends Actor { + var messageNumber: Int = 0 + + protected def receive = { + case _ ⇒ + val payload = "%s".format(messageNumber) + messageNumber = messageNumber + 1 + actorRef ! ZMQMessage(payload.getBytes) + } + } + + object FreePort { + + def isPortFree(port: Int) = { + try { + val socket = new Socket("127.0.0.1", port) + socket.close() + false + } catch { + case e: ConnectException ⇒ true + case e: SocketException if e.getMessage == "Connection reset by peer" ⇒ true + } + } + + private def newPort = Random.nextInt(55365) + 10000 + + def randomFreePort(maxRetries: Int = 50) = { + var count = 0 + var freePort = newPort + while (!isPortFree(freePort)) { + freePort = newPort + count += 1 + if (count >= maxRetries) { + throw new RuntimeException("Couldn't determine a free port") + } + } + freePort + } + } +} diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 091346de34..9d45b1d888 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -195,6 +195,17 @@ object AkkaBuild extends Build { ) ) + + lazy val zeroMQ = Project( + id = "akka-zeromq", + base = file("akka-zeromq"), + dependencies = Seq(actor, testkit % "test;test->test"), + settings = defaultSettings ++ Seq( + libraryDependencies ++= Dependencies.zeroMQ + ) + ) + + // lazy val spring = Project( // id = "akka-spring", // base = file("akka-spring"), @@ -434,6 +445,8 @@ object Dependencies { val tutorials = Seq(Test.scalatest, Test.junit) val docs = Seq(Test.scalatest, Test.junit) + + val zeroMQ = Seq(Test.scalatest, Test.junit, protobuf, Dependency.zeroMQ) } object Dependency { @@ -489,6 +502,7 @@ object Dependency { val zkClient = "zkclient" % "zkclient" % "0.3" // ApacheV2 val zookeeper = "org.apache.hadoop.zookeeper" % "zookeeper" % V.Zookeeper // ApacheV2 val zookeeperLock = "org.apache.hadoop.zookeeper" % "zookeeper-recipes-lock" % V.Zookeeper // ApacheV2 + val zeroMQ = "org.zeromq" %% "zeromq-scala-binding" % "0.0.3" // ApacheV2 // Provided From 448ccadb7c7841ae77bef14d59a34db33a6c56c5 Mon Sep 17 00:00:00 2001 From: Ivan Porto Carrero Date: Sat, 14 Jan 2012 11:46:51 +0100 Subject: [PATCH 025/152] specs pass again --- akka-actor/src/main/resources/reference.conf | 2 +- .../akka/zeromq/ConcurrentSocketActor.scala | 23 ++++++++++--------- .../scala/akka/zeromq/ZeroMQExtension.scala | 4 ++-- .../zeromq/ConcurrentSocketActorSpec.scala | 23 +++++++++++++------ 4 files changed, 31 insertions(+), 21 deletions(-) diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index 68b614ed28..c496f48913 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -146,7 +146,7 @@ akka { } } - zeromq-dispatcher { + zmqdispatcher { # A zeromq socket needs to be pinned to the thread that created it. # Changing this value results in weird errors and race conditions within zeromq type = "PinnedDispatcher" diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala index a1a340e282..735b7f6f83 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala @@ -6,7 +6,8 @@ package akka.zeromq import org.zeromq.ZMQ.{ Socket, Poller } import org.zeromq.{ ZMQ ⇒ JZMQ } import akka.actor._ -import akka.dispatch.{ Promise, Dispatchers, Future } +import akka.dispatch.{ Await, Promise, Dispatchers, Future } +import akka.util.duration._ private[zeromq] sealed trait PollLifeCycle private[zeromq] case object NoResults extends PollLifeCycle @@ -137,7 +138,7 @@ private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Ac override def postStop { currentPoll foreach { _ complete Right(Closing) } poller.unregister(socket) - socket.close + if (socket != null) socket.close notifyListener(Closed) } @@ -155,13 +156,13 @@ private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Ac private var currentPoll: Option[Promise[PollLifeCycle]] = None private def pollAndReceiveFrames() { - currentPoll = currentPoll orElse Some(newEventLoop) + currentPoll = currentPoll orElse newEventLoop } - private def newEventLoop: Promise[PollLifeCycle] = { + private def newEventLoop: Option[Promise[PollLifeCycle]] = if (poller.getSize > 0) { implicit val executor = context.system.dispatchers.defaultGlobalDispatcher - (Future { - if (poller.poll(params.pollTimeoutDuration.toMillis) > 0 && poller.pollin(0)) Results else NoResults + Some((Future { + if (poller.poll(params.pollTimeoutDuration.toMicros) > 0 && poller.pollin(0)) Results else NoResults }).asInstanceOf[Promise[PollLifeCycle]] onSuccess { case Results ⇒ if (!self.isTerminated) self ! 'receiveFrames case NoResults ⇒ if (!self.isTerminated) self ! 'poll @@ -173,15 +174,15 @@ private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Ac } if (!self.isTerminated) self ! 'poll } - } - } + }) + } else None private def receiveFrames(): Seq[Frame] = { @inline def receiveBytes(): Array[Byte] = socket.recv(0) match { - case null ⇒ noBytes - case bytes: Array[Byte] if bytes.length > 0 ⇒ bytes - case _ ⇒ noBytes + case null ⇒ noBytes + case bytes: Array[_] if bytes.length > 0 ⇒ bytes + case _ ⇒ noBytes } receiveBytes() match { diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala index 2515647050..81cb057417 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala @@ -42,11 +42,11 @@ class ZeroMQExtension(system: ActorSystem) extends Extension { listener: Option[ActorRef] = None, context: Context = DefaultContext, // For most applications you want to use the default context deserializer: Deserializer = new ZMQMessageDeserializer, - pollTimeoutDuration: Duration = 100 millis) = { + pollTimeoutDuration: Duration = 500 millis) = { verifyZeroMQVersion val params = SocketParameters(socketType, context, listener, deserializer, pollTimeoutDuration) implicit val timeout = system.settings.ActorTimeout - val req = (zeromq ? Props(new ConcurrentSocketActor(params)).withDispatcher("zmqdispatcher")).mapTo[ActorRef] + val req = (zeromq ? Props(new ConcurrentSocketActor(params)).withDispatcher("akka.actor.zmqdispatcher")).mapTo[ActorRef] Await.result(req, timeout.duration) } diff --git a/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala b/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala index 3b7af45196..9552554a19 100644 --- a/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala +++ b/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala @@ -5,11 +5,11 @@ package akka.zeromq import org.scalatest.matchers.MustMatchers import akka.testkit.{ TestProbe, DefaultTimeout, AkkaSpec } -import akka.actor.{ Actor, Props, ActorRef } import akka.util.Timeout import akka.util.duration._ import java.net.{ SocketException, ConnectException, Socket } import util.Random +import akka.actor.{ Cancellable, Actor, Props, ActorRef } object ConcurrentSocketActorSpec { val config = """ @@ -38,22 +38,18 @@ class ConcurrentSocketActorSpec extends AkkaSpec(ConcurrentSocketActorSpec.confi val publisher = newPublisher(context, publisherProbe.ref) val subscriber = newSubscriber(context, subscriberProbe.ref) val msgGenerator = newMessageGenerator(publisher) - try { subscriberProbe.expectMsg(Connecting) val msgNumbers = subscriberProbe.receiveWhile(2 seconds) { - case msg: ZMQMessage ⇒ { - println("RECV: " + msg.firstFrameAsString) - msg - } + case msg: ZMQMessage ⇒ msg }.map(_.firstFrameAsString.toInt) msgNumbers.length must be > 0 msgNumbers must equal(for (i ← msgNumbers.head to msgNumbers.last) yield i) } finally { system stop msgGenerator within(2 seconds) { awaitCond(msgGenerator.isTerminated) } - system stop subscriber system stop publisher + system stop subscriber subscriberProbe.receiveWhile(1 seconds) { case msg ⇒ msg }.last must equal(Closed) @@ -110,6 +106,19 @@ class ConcurrentSocketActorSpec extends AkkaSpec(ConcurrentSocketActorSpec.confi class MessageGeneratorActor(actorRef: ActorRef) extends Actor { var messageNumber: Int = 0 + private var genMessages: Cancellable = null + + override def preStart() = { + genMessages = system.scheduler.schedule(10 millis, 10 millis, self, 'm) + } + + override def postStop() = { + if (genMessages != null && !genMessages.isCancelled) { + genMessages.cancel + genMessages = null + } + } + protected def receive = { case _ ⇒ val payload = "%s".format(messageNumber) From d2fa14abc32d691807d2e5602ed2f763a0b450bc Mon Sep 17 00:00:00 2001 From: Ivan Porto Carrero Date: Sat, 14 Jan 2012 12:13:46 +0100 Subject: [PATCH 026/152] Attempt to avoid some race condition in zeromq initialization --- .../main/scala/akka/zeromq/ConcurrentSocketActor.scala | 9 ++++----- .../scala/akka/zeromq/ConcurrentSocketActorSpec.scala | 9 +++++++-- 2 files changed, 11 insertions(+), 7 deletions(-) diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala index 735b7f6f83..c1b74b664d 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala @@ -6,8 +6,7 @@ package akka.zeromq import org.zeromq.ZMQ.{ Socket, Poller } import org.zeromq.{ ZMQ ⇒ JZMQ } import akka.actor._ -import akka.dispatch.{ Await, Promise, Dispatchers, Future } -import akka.util.duration._ +import akka.dispatch.{ Promise, Future } private[zeromq] sealed trait PollLifeCycle private[zeromq] case object NoResults extends PollLifeCycle @@ -136,8 +135,8 @@ private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Ac } override def postStop { - currentPoll foreach { _ complete Right(Closing) } poller.unregister(socket) + currentPoll foreach { _ complete Right(Closing) } if (socket != null) socket.close notifyListener(Closed) } @@ -159,7 +158,7 @@ private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Ac currentPoll = currentPoll orElse newEventLoop } - private def newEventLoop: Option[Promise[PollLifeCycle]] = if (poller.getSize > 0) { + private def newEventLoop: Option[Promise[PollLifeCycle]] = { implicit val executor = context.system.dispatchers.defaultGlobalDispatcher Some((Future { if (poller.poll(params.pollTimeoutDuration.toMicros) > 0 && poller.pollin(0)) Results else NoResults @@ -175,7 +174,7 @@ private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Ac if (!self.isTerminated) self ! 'poll } }) - } else None + } private def receiveFrames(): Seq[Frame] = { diff --git a/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala b/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala index 9552554a19..ddf0508ecb 100644 --- a/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala +++ b/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala @@ -24,7 +24,10 @@ akka { """ } -class ConcurrentSocketActorSpec extends AkkaSpec(ConcurrentSocketActorSpec.config) with MustMatchers with DefaultTimeout { +class ConcurrentSocketActorSpec + extends AkkaSpec(ConcurrentSocketActorSpec.config) + with MustMatchers + with DefaultTimeout { val endpoint = "tcp://127.0.0.1:%s" format FreePort.randomFreePort() @@ -38,6 +41,7 @@ class ConcurrentSocketActorSpec extends AkkaSpec(ConcurrentSocketActorSpec.confi val publisher = newPublisher(context, publisherProbe.ref) val subscriber = newSubscriber(context, subscriberProbe.ref) val msgGenerator = newMessageGenerator(publisher) + try { subscriberProbe.expectMsg(Connecting) val msgNumbers = subscriberProbe.receiveWhile(2 seconds) { @@ -85,6 +89,7 @@ class ConcurrentSocketActorSpec extends AkkaSpec(ConcurrentSocketActorSpec.confi } def newMessageGenerator(actorRef: ActorRef) = { system.actorOf(Props(new MessageGeneratorActor(actorRef)).withTimeout(Timeout(10 millis))) + } def checkZeroMQInstallation = try { zmq.version match { @@ -109,7 +114,7 @@ class ConcurrentSocketActorSpec extends AkkaSpec(ConcurrentSocketActorSpec.confi private var genMessages: Cancellable = null override def preStart() = { - genMessages = system.scheduler.schedule(10 millis, 10 millis, self, 'm) + genMessages = system.scheduler.schedule(100 millis, 10 millis, self, 'm) } override def postStop() = { From a2c397d9aacb97bbb17362f9014f1690e2047ce9 Mon Sep 17 00:00:00 2001 From: Ivan Porto Carrero Date: Mon, 16 Jan 2012 00:44:09 +0100 Subject: [PATCH 027/152] Updates style, adds version to error message, makes poll dispatcher configurable and doesn't use an unsupported internal api --- .../akka/zeromq/ConcurrentSocketActor.scala | 194 ++++++++---------- .../src/main/scala/akka/zeromq/Requests.scala | 21 +- .../scala/akka/zeromq/ZeroMQExtension.scala | 17 +- 3 files changed, 104 insertions(+), 128 deletions(-) diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala index c1b74b664d..514c8a0593 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala @@ -18,133 +18,118 @@ private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Ac private val noBytes = Array[Byte]() private val socket: Socket = params.context.socket(params.socketType) private val poller: Poller = params.context.poller - - override def receive: Receive = { - case Send(frames) ⇒ + private object Poll + private object ReceiveFrames + private object ClearPoll + private case class PollError(ex: Throwable) + + private def handleConnectionMessages: Receive = { + case Send(frames) ⇒ { sendFrames(frames) pollAndReceiveFrames() - case ZMQMessage(frames) ⇒ + } + case ZMQMessage(frames) ⇒ { sendFrames(frames) pollAndReceiveFrames() - case Connect(endpoint) ⇒ + } + case Connect(endpoint) ⇒ { socket.connect(endpoint) notifyListener(Connecting) pollAndReceiveFrames() - case Bind(endpoint) ⇒ + } + case Bind(endpoint) ⇒ { socket.bind(endpoint) pollAndReceiveFrames() - case Subscribe(topic) ⇒ + } + case Subscribe(topic) ⇒ { socket.subscribe(topic.toArray) pollAndReceiveFrames() - case Unsubscribe(topic) ⇒ + } + case Unsubscribe(topic) ⇒ { socket.unsubscribe(topic.toArray) pollAndReceiveFrames() - case Linger(value) ⇒ - socket.setLinger(value) - case Linger ⇒ - sender ! socket.getLinger - case ReconnectIVL ⇒ - sender ! socket.getReconnectIVL - case ReconnectIVL(value) ⇒ - socket.setReconnectIVL(value) - case Backlog ⇒ - sender ! socket.getBacklog - case Backlog(value) ⇒ - socket.setBacklog(value) - case ReconnectIVLMax ⇒ - sender ! socket.getReconnectIVLMax - case ReconnectIVLMax(value) ⇒ - socket.setReconnectIVLMax(value) - case MaxMsgSize ⇒ - sender ! socket.getMaxMsgSize - case MaxMsgSize(value) ⇒ - socket.setMaxMsgSize(value) - case SndHWM ⇒ - sender ! socket.getSndHWM - case SndHWM(value) ⇒ - socket.setSndHWM(value) - case RcvHWM ⇒ - sender ! socket.getRcvHWM - case RcvHWM(value) ⇒ - socket.setRcvHWM(value) - case HWM(value) ⇒ - socket.setHWM(value) - case Swap ⇒ - sender ! socket.getSwap - case Swap(value) ⇒ - socket.setSwap(value) - case Affinity ⇒ - sender ! socket.getAffinity - case Affinity(value) ⇒ - socket.setAffinity(value) - case Identity ⇒ - sender ! socket.getIdentity - case Identity(value) ⇒ - socket.setIdentity(value) - case Rate ⇒ - sender ! socket.getRate - case Rate(value) ⇒ - socket.setRate(value) - case RecoveryInterval ⇒ - sender ! socket.getRecoveryInterval - case RecoveryInterval(value) ⇒ - socket.setRecoveryInterval(value) - case MulticastLoop ⇒ - sender ! socket.hasMulticastLoop - case MulticastLoop(value) ⇒ - socket.setMulticastLoop(value) - case MulticastHops ⇒ - sender ! socket.getMulticastHops - case MulticastHops(value) ⇒ - socket.setMulticastHops(value) - case ReceiveTimeOut ⇒ - sender ! socket.getReceiveTimeOut - case ReceiveTimeOut(value) ⇒ - socket.setReceiveTimeOut(value) - case SendTimeOut ⇒ - sender ! socket.getSendTimeOut - case SendTimeOut(value) ⇒ - socket.setSendTimeOut(value) - case SendBufferSize ⇒ - sender ! socket.getSendBufferSize - case SendBufferSize(value) ⇒ - socket.setSendBufferSize(value) - case ReceiveBufferSize ⇒ - sender ! socket.getReceiveBufferSize - case ReceiveBufferSize(value) ⇒ - socket.setReceiveBufferSize(value) - case ReceiveMore ⇒ - sender ! socket.hasReceiveMore - case FileDescriptor ⇒ - sender ! socket.getFD - case 'poll ⇒ { + } + } + + private def handleSocketOption: Receive = { + case Linger(value) ⇒ socket.setLinger(value) + case Linger ⇒ sender ! socket.getLinger + case ReconnectIVL ⇒ sender ! socket.getReconnectIVL + case ReconnectIVL(value) ⇒ socket.setReconnectIVL(value) + case Backlog ⇒ sender ! socket.getBacklog + case Backlog(value) ⇒ socket.setBacklog(value) + case ReconnectIVLMax ⇒ sender ! socket.getReconnectIVLMax + case ReconnectIVLMax(value) ⇒ socket.setReconnectIVLMax(value) + case MaxMsgSize ⇒ sender ! socket.getMaxMsgSize + case MaxMsgSize(value) ⇒ socket.setMaxMsgSize(value) + case SndHWM ⇒ sender ! socket.getSndHWM + case SndHWM(value) ⇒ socket.setSndHWM(value) + case RcvHWM ⇒ sender ! socket.getRcvHWM + case RcvHWM(value) ⇒ socket.setRcvHWM(value) + case HWM(value) ⇒ socket.setHWM(value) + case Swap ⇒ sender ! socket.getSwap + case Swap(value) ⇒ socket.setSwap(value) + case Affinity ⇒ sender ! socket.getAffinity + case Affinity(value) ⇒ socket.setAffinity(value) + case Identity ⇒ sender ! socket.getIdentity + case Identity(value) ⇒ socket.setIdentity(value) + case Rate ⇒ sender ! socket.getRate + case Rate(value) ⇒ socket.setRate(value) + case RecoveryInterval ⇒ sender ! socket.getRecoveryInterval + case RecoveryInterval(value) ⇒ socket.setRecoveryInterval(value) + case MulticastLoop ⇒ sender ! socket.hasMulticastLoop + case MulticastLoop(value) ⇒ socket.setMulticastLoop(value) + case MulticastHops ⇒ sender ! socket.getMulticastHops + case MulticastHops(value) ⇒ socket.setMulticastHops(value) + case ReceiveTimeOut ⇒ sender ! socket.getReceiveTimeOut + case ReceiveTimeOut(value) ⇒ socket.setReceiveTimeOut(value) + case SendTimeOut ⇒ sender ! socket.getSendTimeOut + case SendTimeOut(value) ⇒ socket.setSendTimeOut(value) + case SendBufferSize ⇒ sender ! socket.getSendBufferSize + case SendBufferSize(value) ⇒ socket.setSendBufferSize(value) + case ReceiveBufferSize ⇒ sender ! socket.getReceiveBufferSize + case ReceiveBufferSize(value) ⇒ socket.setReceiveBufferSize(value) + case ReceiveMore ⇒ sender ! socket.hasReceiveMore + case FileDescriptor ⇒ sender ! socket.getFD + } + + private def internalMessage: Receive = { + case Poll ⇒ { currentPoll = None pollAndReceiveFrames() } - case 'receiveFrames ⇒ { + case ReceiveFrames ⇒ { receiveFrames() match { case Seq() ⇒ case frames ⇒ notifyListener(params.deserializer(frames)) } - self ! 'poll + self ! Poll + } + case ClearPoll => currentPoll = None + case PollError(ex) => { + context.system.log.error(ex, "There was a problem polling the zeromq socket") + self ! Poll } } + override def receive: Receive = handleConnectionMessages orElse handleSocketOption orElse internalMessage + override def preStart { poller.register(socket, Poller.POLLIN) } override def postStop { - poller.unregister(socket) - currentPoll foreach { _ complete Right(Closing) } - if (socket != null) socket.close - notifyListener(Closed) + try { + poller.unregister(socket) + currentPoll foreach { _ complete Right(Closing) } + if (socket != null) socket.close + } finally { + notifyListener(Closed) + } } private def sendFrames(frames: Seq[Frame]) { - def sendBytes(bytes: Seq[Byte], flags: Int) { - socket.send(bytes.toArray, flags) - } + def sendBytes(bytes: Seq[Byte], flags: Int) = socket.send(bytes.toArray, flags) val iter = frames.iterator while (iter.hasNext) { val payload = iter.next.payload @@ -155,24 +140,19 @@ private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Ac private var currentPoll: Option[Promise[PollLifeCycle]] = None private def pollAndReceiveFrames() { - currentPoll = currentPoll orElse newEventLoop + if (currentPoll.isEmpty) currentPoll = newEventLoop } private def newEventLoop: Option[Promise[PollLifeCycle]] = { - implicit val executor = context.system.dispatchers.defaultGlobalDispatcher + implicit val executor = params.pollDispatcher getOrElse context.system.dispatchers.defaultGlobalDispatcher Some((Future { if (poller.poll(params.pollTimeoutDuration.toMicros) > 0 && poller.pollin(0)) Results else NoResults }).asInstanceOf[Promise[PollLifeCycle]] onSuccess { - case Results ⇒ if (!self.isTerminated) self ! 'receiveFrames - case NoResults ⇒ if (!self.isTerminated) self ! 'poll - case _ ⇒ currentPoll = None + case Results ⇒ self ! ReceiveFrames + case NoResults ⇒ self ! Poll + case _ ⇒ self ! ClearPoll } onFailure { - case ex ⇒ { - if (context.system != null) { - context.system.log.error(ex, "There was an error receiving messages on the zeromq socket") - } - if (!self.isTerminated) self ! 'poll - } + case ex ⇒ self ! PollError(ex) }) } diff --git a/akka-zeromq/src/main/scala/akka/zeromq/Requests.scala b/akka-zeromq/src/main/scala/akka/zeromq/Requests.scala index a1ce8f6c5e..3cb5388069 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/Requests.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/Requests.scala @@ -15,32 +15,22 @@ private[zeromq] case object Close extends Request case class Subscribe(payload: Seq[Byte]) extends Request object Subscribe { - def apply(topic: String): Subscribe = { - Subscribe(topic.getBytes) - } + def apply(topic: String): Subscribe = Subscribe(topic.getBytes) } case class Unsubscribe(payload: Seq[Byte]) extends Request object Unsubscribe { - def apply(topic: String): Unsubscribe = { - Unsubscribe(topic.getBytes) - } + def apply(topic: String): Unsubscribe = Unsubscribe(topic.getBytes) } case class Send(frames: Seq[Frame]) extends Request case class ZMQMessage(frames: Seq[Frame]) { - def firstFrameAsString = { - new String(frames.head.payload.toArray) - } + def firstFrameAsString = new String(frames.head.payload.toArray) } object ZMQMessage { - def apply(bytes: Array[Byte]): ZMQMessage = { - ZMQMessage(Seq(Frame(bytes))) - } - def apply(message: Message): ZMQMessage = { - ZMQMessage(message.toByteArray) - } + def apply(bytes: Array[Byte]): ZMQMessage = ZMQMessage(Seq(Frame(bytes))) + def apply(message: Message): ZMQMessage = ZMQMessage(message.toByteArray) } case class Linger(value: Long) extends SocketOption @@ -65,7 +55,6 @@ case class RcvHWM(value: Long) extends SocketOption object RcvHWM extends SocketOptionQuery case class HWM(value: Long) extends SocketOption -/* object HWM extends SocketOptionQuery */ case class Swap(value: Long) extends SocketOption object Swap extends SocketOptionQuery diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala index 81cb057417..91d86ebb05 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala @@ -8,12 +8,13 @@ import akka.util.duration._ import akka.zeromq.SocketType._ import org.zeromq.{ ZMQ ⇒ JZMQ } import akka.actor._ -import akka.dispatch.Await +import akka.dispatch.{Dispatcher, Await} case class SocketParameters( socketType: SocketType, context: Context, listener: Option[ActorRef] = None, + pollDispatcher: Option[Dispatcher] = None, deserializer: Deserializer = new ZMQMessageDeserializer, pollTimeoutDuration: Duration = 100 millis) @@ -24,6 +25,9 @@ case class ZeroMQVersion(major: Int, minor: Int, patch: Int) { object ZeroMQExtension extends ExtensionId[ZeroMQExtension] with ExtensionIdProvider { def lookup() = this def createExtension(system: ActorSystemImpl) = new ZeroMQExtension(system) + + private val minVersionString = "2.1.0" + private val minVersion = JZMQ.makeVersion(2, 1, 0) } class ZeroMQExtension(system: ActorSystem) extends Extension { @@ -42,9 +46,10 @@ class ZeroMQExtension(system: ActorSystem) extends Extension { listener: Option[ActorRef] = None, context: Context = DefaultContext, // For most applications you want to use the default context deserializer: Deserializer = new ZMQMessageDeserializer, + pollDispatcher: Option[Dispatcher] = None, pollTimeoutDuration: Duration = 500 millis) = { verifyZeroMQVersion - val params = SocketParameters(socketType, context, listener, deserializer, pollTimeoutDuration) + val params = SocketParameters(socketType, context, listener, pollDispatcher, deserializer, pollTimeoutDuration) implicit val timeout = system.settings.ActorTimeout val req = (zeromq ? Props(new ConcurrentSocketActor(params)).withDispatcher("akka.actor.zmqdispatcher")).mapTo[ActorRef] Await.result(req, timeout.duration) @@ -52,14 +57,16 @@ class ZeroMQExtension(system: ActorSystem) extends Extension { val zeromq: ActorRef = { verifyZeroMQVersion - system.asInstanceOf[ActorSystemImpl].systemActorOf(Props(new Actor { + system.actorOf(Props(new Actor { protected def receive = { case p: Props ⇒ sender ! context.actorOf(p) } }), "zeromq") } + + private def verifyZeroMQVersion = { require( - JZMQ.getFullVersion > JZMQ.makeVersion(2, 1, 0), - "Unsupported ZeroMQ version: %s".format(JZMQ.getVersionString)) + JZMQ.getFullVersion > ZeroMQExtension.minVersion, + "Unsupported ZeroMQ version: %s, akka needs at least: %s".format(JZMQ.getVersionString, ZeroMQExtension.minVersionString)) } } From d08d92246b834d7f1d2bbe1f4165128971545750 Mon Sep 17 00:00:00 2001 From: Ivan Porto Carrero Date: Mon, 16 Jan 2012 00:47:30 +0100 Subject: [PATCH 028/152] Also actually type case object if you want a case object --- .../src/main/scala/akka/zeromq/ConcurrentSocketActor.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala index 514c8a0593..1be4e21f80 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala @@ -18,9 +18,9 @@ private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Ac private val noBytes = Array[Byte]() private val socket: Socket = params.context.socket(params.socketType) private val poller: Poller = params.context.poller - private object Poll - private object ReceiveFrames - private object ClearPoll + private case object Poll + private case object ReceiveFrames + private case object ClearPoll private case class PollError(ex: Throwable) private def handleConnectionMessages: Receive = { From 3fd10205b570ed0cde0a9ad7bc81ee7f76f142b3 Mon Sep 17 00:00:00 2001 From: Ivan Porto Carrero Date: Mon, 16 Jan 2012 10:48:23 +0100 Subject: [PATCH 029/152] Move reference.conf values, use instance logger --- akka-actor/src/main/resources/reference.conf | 6 ------ akka-zeromq/src/main/resources/reference.conf | 18 ++++++++++++++++++ .../akka/zeromq/ConcurrentSocketActor.scala | 7 +++++-- .../scala/akka/zeromq/ZeroMQExtension.scala | 2 +- .../zeromq/ConcurrentSocketActorSpec.scala | 4 ++-- 5 files changed, 26 insertions(+), 11 deletions(-) create mode 100644 akka-zeromq/src/main/resources/reference.conf diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index c496f48913..94efc34176 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -146,12 +146,6 @@ akka { } } - zmqdispatcher { - # A zeromq socket needs to be pinned to the thread that created it. - # Changing this value results in weird errors and race conditions within zeromq - type = "PinnedDispatcher" - } - default-dispatcher { # Must be one of the following # Dispatcher, (BalancingDispatcher, only valid when all actors using it are of diff --git a/akka-zeromq/src/main/resources/reference.conf b/akka-zeromq/src/main/resources/reference.conf new file mode 100644 index 0000000000..ce562b9449 --- /dev/null +++ b/akka-zeromq/src/main/resources/reference.conf @@ -0,0 +1,18 @@ +############################## +# Akka Reference Config File # +############################## + +# This the reference config file has all the default settings. +# Make your edits/overrides in your application.conf. + +akka { + + zeromq { + + socket-dispatcher { + # A zeromq socket needs to be pinned to the thread that created it. + # Changing this value results in weird errors and race conditions within zeromq + type = "PinnedDispatcher" + } + } +} diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala index 1be4e21f80..fc71400fdd 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala @@ -7,6 +7,7 @@ import org.zeromq.ZMQ.{ Socket, Poller } import org.zeromq.{ ZMQ ⇒ JZMQ } import akka.actor._ import akka.dispatch.{ Promise, Future } +import akka.event.Logging private[zeromq] sealed trait PollLifeCycle private[zeromq] case object NoResults extends PollLifeCycle @@ -18,6 +19,8 @@ private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Ac private val noBytes = Array[Byte]() private val socket: Socket = params.context.socket(params.socketType) private val poller: Poller = params.context.poller + private val log = Logging(context.system, this) + private case object Poll private case object ReceiveFrames private case object ClearPoll @@ -107,7 +110,7 @@ private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Ac } case ClearPoll => currentPoll = None case PollError(ex) => { - context.system.log.error(ex, "There was a problem polling the zeromq socket") + log.error(ex, "There was a problem polling the zeromq socket") self ! Poll } } @@ -144,7 +147,7 @@ private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Ac } private def newEventLoop: Option[Promise[PollLifeCycle]] = { - implicit val executor = params.pollDispatcher getOrElse context.system.dispatchers.defaultGlobalDispatcher + implicit val executor = params.pollDispatcher getOrElse context.system.dispatcher Some((Future { if (poller.poll(params.pollTimeoutDuration.toMicros) > 0 && poller.pollin(0)) Results else NoResults }).asInstanceOf[Promise[PollLifeCycle]] onSuccess { diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala index 91d86ebb05..06b929e48a 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala @@ -51,7 +51,7 @@ class ZeroMQExtension(system: ActorSystem) extends Extension { verifyZeroMQVersion val params = SocketParameters(socketType, context, listener, pollDispatcher, deserializer, pollTimeoutDuration) implicit val timeout = system.settings.ActorTimeout - val req = (zeromq ? Props(new ConcurrentSocketActor(params)).withDispatcher("akka.actor.zmqdispatcher")).mapTo[ActorRef] + val req = (zeromq ? Props(new ConcurrentSocketActor(params)).withDispatcher("akka.zeromq.socket-dispatcher")).mapTo[ActorRef] Await.result(req, timeout.duration) } diff --git a/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala b/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala index ddf0508ecb..6c256dd58e 100644 --- a/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala +++ b/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala @@ -15,8 +15,8 @@ object ConcurrentSocketActorSpec { val config = """ akka { extensions = ["akka.zeromq.ZeroMQExtension$"] - actor { - zmqdispatcher { + zeromq { + socket-dispatcher { type = "PinnedDispatcher" } } From 1d27d26022be04458e5eed13804fb1742e5aa4ef Mon Sep 17 00:00:00 2001 From: Ivan Porto Carrero Date: Mon, 16 Jan 2012 10:54:46 +0100 Subject: [PATCH 030/152] Adds explicit return type for newSocket, removes protected receive and makes context a val --- .../akka/zeromq/ConcurrentSocketActor.scala | 88 +++++++++---------- .../src/main/scala/akka/zeromq/Context.scala | 2 +- .../scala/akka/zeromq/ZeroMQExtension.scala | 10 +-- 3 files changed, 49 insertions(+), 51 deletions(-) diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala index fc71400fdd..723e7ef72f 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala @@ -25,7 +25,7 @@ private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Ac private case object ReceiveFrames private case object ClearPoll private case class PollError(ex: Throwable) - + private def handleConnectionMessages: Receive = { case Send(frames) ⇒ { sendFrames(frames) @@ -51,51 +51,51 @@ private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Ac case Unsubscribe(topic) ⇒ { socket.unsubscribe(topic.toArray) pollAndReceiveFrames() - } + } } - + private def handleSocketOption: Receive = { - case Linger(value) ⇒ socket.setLinger(value) - case Linger ⇒ sender ! socket.getLinger - case ReconnectIVL ⇒ sender ! socket.getReconnectIVL - case ReconnectIVL(value) ⇒ socket.setReconnectIVL(value) - case Backlog ⇒ sender ! socket.getBacklog - case Backlog(value) ⇒ socket.setBacklog(value) - case ReconnectIVLMax ⇒ sender ! socket.getReconnectIVLMax - case ReconnectIVLMax(value) ⇒ socket.setReconnectIVLMax(value) - case MaxMsgSize ⇒ sender ! socket.getMaxMsgSize - case MaxMsgSize(value) ⇒ socket.setMaxMsgSize(value) - case SndHWM ⇒ sender ! socket.getSndHWM - case SndHWM(value) ⇒ socket.setSndHWM(value) - case RcvHWM ⇒ sender ! socket.getRcvHWM - case RcvHWM(value) ⇒ socket.setRcvHWM(value) - case HWM(value) ⇒ socket.setHWM(value) - case Swap ⇒ sender ! socket.getSwap - case Swap(value) ⇒ socket.setSwap(value) - case Affinity ⇒ sender ! socket.getAffinity - case Affinity(value) ⇒ socket.setAffinity(value) - case Identity ⇒ sender ! socket.getIdentity - case Identity(value) ⇒ socket.setIdentity(value) - case Rate ⇒ sender ! socket.getRate - case Rate(value) ⇒ socket.setRate(value) - case RecoveryInterval ⇒ sender ! socket.getRecoveryInterval - case RecoveryInterval(value) ⇒ socket.setRecoveryInterval(value) - case MulticastLoop ⇒ sender ! socket.hasMulticastLoop - case MulticastLoop(value) ⇒ socket.setMulticastLoop(value) - case MulticastHops ⇒ sender ! socket.getMulticastHops - case MulticastHops(value) ⇒ socket.setMulticastHops(value) - case ReceiveTimeOut ⇒ sender ! socket.getReceiveTimeOut - case ReceiveTimeOut(value) ⇒ socket.setReceiveTimeOut(value) - case SendTimeOut ⇒ sender ! socket.getSendTimeOut - case SendTimeOut(value) ⇒ socket.setSendTimeOut(value) - case SendBufferSize ⇒ sender ! socket.getSendBufferSize - case SendBufferSize(value) ⇒ socket.setSendBufferSize(value) - case ReceiveBufferSize ⇒ sender ! socket.getReceiveBufferSize + case Linger(value) ⇒ socket.setLinger(value) + case Linger ⇒ sender ! socket.getLinger + case ReconnectIVL ⇒ sender ! socket.getReconnectIVL + case ReconnectIVL(value) ⇒ socket.setReconnectIVL(value) + case Backlog ⇒ sender ! socket.getBacklog + case Backlog(value) ⇒ socket.setBacklog(value) + case ReconnectIVLMax ⇒ sender ! socket.getReconnectIVLMax + case ReconnectIVLMax(value) ⇒ socket.setReconnectIVLMax(value) + case MaxMsgSize ⇒ sender ! socket.getMaxMsgSize + case MaxMsgSize(value) ⇒ socket.setMaxMsgSize(value) + case SndHWM ⇒ sender ! socket.getSndHWM + case SndHWM(value) ⇒ socket.setSndHWM(value) + case RcvHWM ⇒ sender ! socket.getRcvHWM + case RcvHWM(value) ⇒ socket.setRcvHWM(value) + case HWM(value) ⇒ socket.setHWM(value) + case Swap ⇒ sender ! socket.getSwap + case Swap(value) ⇒ socket.setSwap(value) + case Affinity ⇒ sender ! socket.getAffinity + case Affinity(value) ⇒ socket.setAffinity(value) + case Identity ⇒ sender ! socket.getIdentity + case Identity(value) ⇒ socket.setIdentity(value) + case Rate ⇒ sender ! socket.getRate + case Rate(value) ⇒ socket.setRate(value) + case RecoveryInterval ⇒ sender ! socket.getRecoveryInterval + case RecoveryInterval(value) ⇒ socket.setRecoveryInterval(value) + case MulticastLoop ⇒ sender ! socket.hasMulticastLoop + case MulticastLoop(value) ⇒ socket.setMulticastLoop(value) + case MulticastHops ⇒ sender ! socket.getMulticastHops + case MulticastHops(value) ⇒ socket.setMulticastHops(value) + case ReceiveTimeOut ⇒ sender ! socket.getReceiveTimeOut + case ReceiveTimeOut(value) ⇒ socket.setReceiveTimeOut(value) + case SendTimeOut ⇒ sender ! socket.getSendTimeOut + case SendTimeOut(value) ⇒ socket.setSendTimeOut(value) + case SendBufferSize ⇒ sender ! socket.getSendBufferSize + case SendBufferSize(value) ⇒ socket.setSendBufferSize(value) + case ReceiveBufferSize ⇒ sender ! socket.getReceiveBufferSize case ReceiveBufferSize(value) ⇒ socket.setReceiveBufferSize(value) - case ReceiveMore ⇒ sender ! socket.hasReceiveMore - case FileDescriptor ⇒ sender ! socket.getFD + case ReceiveMore ⇒ sender ! socket.hasReceiveMore + case FileDescriptor ⇒ sender ! socket.getFD } - + private def internalMessage: Receive = { case Poll ⇒ { currentPoll = None @@ -108,8 +108,8 @@ private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Ac } self ! Poll } - case ClearPoll => currentPoll = None - case PollError(ex) => { + case ClearPoll ⇒ currentPoll = None + case PollError(ex) ⇒ { log.error(ex, "There was a problem polling the zeromq socket") self ! Poll } diff --git a/akka-zeromq/src/main/scala/akka/zeromq/Context.scala b/akka-zeromq/src/main/scala/akka/zeromq/Context.scala index 073270436f..7d768e1492 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/Context.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/Context.scala @@ -7,7 +7,7 @@ import org.zeromq.{ ZMQ ⇒ JZMQ } import akka.zeromq.SocketType._ class Context(numIoThreads: Int) { - private var context = JZMQ.context(numIoThreads) + private val context = JZMQ.context(numIoThreads) def socket(socketType: SocketType) = { context.socket(socketType.id) } diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala index 06b929e48a..af01428ebc 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala @@ -8,7 +8,7 @@ import akka.util.duration._ import akka.zeromq.SocketType._ import org.zeromq.{ ZMQ ⇒ JZMQ } import akka.actor._ -import akka.dispatch.{Dispatcher, Await} +import akka.dispatch.{ Dispatcher, Await } case class SocketParameters( socketType: SocketType, @@ -25,7 +25,7 @@ case class ZeroMQVersion(major: Int, minor: Int, patch: Int) { object ZeroMQExtension extends ExtensionId[ZeroMQExtension] with ExtensionIdProvider { def lookup() = this def createExtension(system: ActorSystemImpl) = new ZeroMQExtension(system) - + private val minVersionString = "2.1.0" private val minVersion = JZMQ.makeVersion(2, 1, 0) } @@ -47,7 +47,7 @@ class ZeroMQExtension(system: ActorSystem) extends Extension { context: Context = DefaultContext, // For most applications you want to use the default context deserializer: Deserializer = new ZMQMessageDeserializer, pollDispatcher: Option[Dispatcher] = None, - pollTimeoutDuration: Duration = 500 millis) = { + pollTimeoutDuration: Duration = 500 millis): ActorRef = { verifyZeroMQVersion val params = SocketParameters(socketType, context, listener, pollDispatcher, deserializer, pollTimeoutDuration) implicit val timeout = system.settings.ActorTimeout @@ -58,11 +58,9 @@ class ZeroMQExtension(system: ActorSystem) extends Extension { val zeromq: ActorRef = { verifyZeroMQVersion system.actorOf(Props(new Actor { - protected def receive = { case p: Props ⇒ sender ! context.actorOf(p) } + def receive = { case p: Props ⇒ sender ! context.actorOf(p) } }), "zeromq") } - - private def verifyZeroMQVersion = { require( From 26a5918b3807d4f420c6c4f511462102de57b48c Mon Sep 17 00:00:00 2001 From: Ivan Porto Carrero Date: Wed, 18 Jan 2012 21:01:14 +0100 Subject: [PATCH 031/152] Use system.dispatchers.lookup --- .../akka/zeromq/ConcurrentSocketActor.scala | 44 ++++++++++--------- .../scala/akka/zeromq/ZeroMQExtension.scala | 4 +- 2 files changed, 26 insertions(+), 22 deletions(-) diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala index 723e7ef72f..ca7cc08e89 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.zeromq @@ -56,42 +56,42 @@ private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Ac private def handleSocketOption: Receive = { case Linger(value) ⇒ socket.setLinger(value) - case Linger ⇒ sender ! socket.getLinger - case ReconnectIVL ⇒ sender ! socket.getReconnectIVL case ReconnectIVL(value) ⇒ socket.setReconnectIVL(value) - case Backlog ⇒ sender ! socket.getBacklog case Backlog(value) ⇒ socket.setBacklog(value) - case ReconnectIVLMax ⇒ sender ! socket.getReconnectIVLMax case ReconnectIVLMax(value) ⇒ socket.setReconnectIVLMax(value) - case MaxMsgSize ⇒ sender ! socket.getMaxMsgSize case MaxMsgSize(value) ⇒ socket.setMaxMsgSize(value) - case SndHWM ⇒ sender ! socket.getSndHWM case SndHWM(value) ⇒ socket.setSndHWM(value) - case RcvHWM ⇒ sender ! socket.getRcvHWM case RcvHWM(value) ⇒ socket.setRcvHWM(value) case HWM(value) ⇒ socket.setHWM(value) - case Swap ⇒ sender ! socket.getSwap case Swap(value) ⇒ socket.setSwap(value) - case Affinity ⇒ sender ! socket.getAffinity case Affinity(value) ⇒ socket.setAffinity(value) - case Identity ⇒ sender ! socket.getIdentity case Identity(value) ⇒ socket.setIdentity(value) - case Rate ⇒ sender ! socket.getRate case Rate(value) ⇒ socket.setRate(value) - case RecoveryInterval ⇒ sender ! socket.getRecoveryInterval case RecoveryInterval(value) ⇒ socket.setRecoveryInterval(value) - case MulticastLoop ⇒ sender ! socket.hasMulticastLoop case MulticastLoop(value) ⇒ socket.setMulticastLoop(value) - case MulticastHops ⇒ sender ! socket.getMulticastHops case MulticastHops(value) ⇒ socket.setMulticastHops(value) - case ReceiveTimeOut ⇒ sender ! socket.getReceiveTimeOut case ReceiveTimeOut(value) ⇒ socket.setReceiveTimeOut(value) - case SendTimeOut ⇒ sender ! socket.getSendTimeOut case SendTimeOut(value) ⇒ socket.setSendTimeOut(value) - case SendBufferSize ⇒ sender ! socket.getSendBufferSize case SendBufferSize(value) ⇒ socket.setSendBufferSize(value) - case ReceiveBufferSize ⇒ sender ! socket.getReceiveBufferSize case ReceiveBufferSize(value) ⇒ socket.setReceiveBufferSize(value) + case Linger ⇒ sender ! socket.getLinger + case ReconnectIVL ⇒ sender ! socket.getReconnectIVL + case Backlog ⇒ sender ! socket.getBacklog + case ReconnectIVLMax ⇒ sender ! socket.getReconnectIVLMax + case MaxMsgSize ⇒ sender ! socket.getMaxMsgSize + case SndHWM ⇒ sender ! socket.getSndHWM + case RcvHWM ⇒ sender ! socket.getRcvHWM + case Swap ⇒ sender ! socket.getSwap + case Affinity ⇒ sender ! socket.getAffinity + case Identity ⇒ sender ! socket.getIdentity + case Rate ⇒ sender ! socket.getRate + case RecoveryInterval ⇒ sender ! socket.getRecoveryInterval + case MulticastLoop ⇒ sender ! socket.hasMulticastLoop + case MulticastHops ⇒ sender ! socket.getMulticastHops + case ReceiveTimeOut ⇒ sender ! socket.getReceiveTimeOut + case SendTimeOut ⇒ sender ! socket.getSendTimeOut + case SendBufferSize ⇒ sender ! socket.getSendBufferSize + case ReceiveBufferSize ⇒ sender ! socket.getReceiveBufferSize case ReceiveMore ⇒ sender ! socket.hasReceiveMore case FileDescriptor ⇒ sender ! socket.getFD } @@ -146,8 +146,12 @@ private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Ac if (currentPoll.isEmpty) currentPoll = newEventLoop } + private val eventLoopDispatcher = { + params.pollDispatcher.map(d ⇒ context.system.dispatchers.lookup(d)) getOrElse context.system.dispatcher + } + private def newEventLoop: Option[Promise[PollLifeCycle]] = { - implicit val executor = params.pollDispatcher getOrElse context.system.dispatcher + implicit val executor = eventLoopDispatcher Some((Future { if (poller.poll(params.pollTimeoutDuration.toMicros) > 0 && poller.pollin(0)) Results else NoResults }).asInstanceOf[Promise[PollLifeCycle]] onSuccess { diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala index af01428ebc..e44bc00d06 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala @@ -14,7 +14,7 @@ case class SocketParameters( socketType: SocketType, context: Context, listener: Option[ActorRef] = None, - pollDispatcher: Option[Dispatcher] = None, + pollDispatcher: Option[String] = None, deserializer: Deserializer = new ZMQMessageDeserializer, pollTimeoutDuration: Duration = 100 millis) @@ -46,7 +46,7 @@ class ZeroMQExtension(system: ActorSystem) extends Extension { listener: Option[ActorRef] = None, context: Context = DefaultContext, // For most applications you want to use the default context deserializer: Deserializer = new ZMQMessageDeserializer, - pollDispatcher: Option[Dispatcher] = None, + pollDispatcher: Option[String] = None, pollTimeoutDuration: Duration = 500 millis): ActorRef = { verifyZeroMQVersion val params = SocketParameters(socketType, context, listener, pollDispatcher, deserializer, pollTimeoutDuration) From 2757869c62d1bd879f23dae1facd942e106aee7b Mon Sep 17 00:00:00 2001 From: Peter Vlugter Date: Wed, 18 Jan 2012 15:59:59 +1300 Subject: [PATCH 032/152] Update to new Java API for Scala STM --- .../docs/transactor/CoordinatedCounter.java | 18 ++--- .../akka/docs/transactor/Coordinator.java | 5 +- .../code/akka/docs/transactor/Counter.java | 12 ++-- .../akka/docs/transactor/FriendlyCounter.java | 12 ++-- akka-docs/java/transactors.rst | 2 +- .../scala/akka/transactor/Atomically.scala | 67 ------------------- .../scala/akka/transactor/Coordinated.scala | 18 +++-- .../akka/transactor/UntypedTransactor.scala | 4 +- .../transactor/UntypedCoordinatedCounter.java | 25 +++---- .../java/akka/transactor/UntypedCounter.java | 23 +++---- .../java/akka/transactor/UntypedFailer.java | 2 +- project/AkkaBuild.scala | 2 +- 12 files changed, 61 insertions(+), 129 deletions(-) delete mode 100644 akka-transactor/src/main/scala/akka/transactor/Atomically.scala diff --git a/akka-docs/java/code/akka/docs/transactor/CoordinatedCounter.java b/akka-docs/java/code/akka/docs/transactor/CoordinatedCounter.java index dca10b8984..f17e86ade0 100644 --- a/akka-docs/java/code/akka/docs/transactor/CoordinatedCounter.java +++ b/akka-docs/java/code/akka/docs/transactor/CoordinatedCounter.java @@ -7,15 +7,11 @@ package akka.docs.transactor; //#class import akka.actor.*; import akka.transactor.*; -import scala.concurrent.stm.*; +import scala.concurrent.stm.Ref; +import static scala.concurrent.stm.JavaAPI.*; public class CoordinatedCounter extends UntypedActor { - private Ref count = Stm.ref(0); - - private void increment(InTxn txn) { - Integer newValue = count.get(txn) + 1; - count.set(newValue, txn); - } + private Ref.View count = newRef(0); public void onReceive(Object incoming) throws Exception { if (incoming instanceof Coordinated) { @@ -26,14 +22,14 @@ public class CoordinatedCounter extends UntypedActor { if (increment.hasFriend()) { increment.getFriend().tell(coordinated.coordinate(new Increment())); } - coordinated.atomic(new Atomically() { - public void atomically(InTxn txn) { - increment(txn); + coordinated.atomic(new Runnable() { + public void run() { + increment(count, 1); } }); } } else if ("GetCount".equals(incoming)) { - getSender().tell(count.single().get()); + getSender().tell(count.get()); } else { unhandled(incoming); } diff --git a/akka-docs/java/code/akka/docs/transactor/Coordinator.java b/akka-docs/java/code/akka/docs/transactor/Coordinator.java index 6854ed99f6..195906f5f6 100644 --- a/akka-docs/java/code/akka/docs/transactor/Coordinator.java +++ b/akka-docs/java/code/akka/docs/transactor/Coordinator.java @@ -6,7 +6,6 @@ package akka.docs.transactor; import akka.actor.*; import akka.transactor.*; -import scala.concurrent.stm.*; public class Coordinator extends UntypedActor { public void onReceive(Object incoming) throws Exception { @@ -15,8 +14,8 @@ public class Coordinator extends UntypedActor { Object message = coordinated.getMessage(); if (message instanceof Message) { //#coordinated-atomic - coordinated.atomic(new Atomically() { - public void atomically(InTxn txn) { + coordinated.atomic(new Runnable() { + public void run() { // do something in the coordinated transaction ... } }); diff --git a/akka-docs/java/code/akka/docs/transactor/Counter.java b/akka-docs/java/code/akka/docs/transactor/Counter.java index 0a6b7b2219..efe2aaed72 100644 --- a/akka-docs/java/code/akka/docs/transactor/Counter.java +++ b/akka-docs/java/code/akka/docs/transactor/Counter.java @@ -6,21 +6,21 @@ package akka.docs.transactor; //#class import akka.transactor.*; -import scala.concurrent.stm.*; +import scala.concurrent.stm.Ref; +import static scala.concurrent.stm.JavaAPI.*; public class Counter extends UntypedTransactor { - Ref count = Stm.ref(0); + Ref.View count = newRef(0); - public void atomically(InTxn txn, Object message) { + public void atomically(Object message) { if (message instanceof Increment) { - Integer newValue = count.get(txn) + 1; - count.set(newValue, txn); + increment(count, 1); } } @Override public boolean normally(Object message) { if ("GetCount".equals(message)) { - getSender().tell(count.single().get()); + getSender().tell(count.get()); return true; } else return false; } diff --git a/akka-docs/java/code/akka/docs/transactor/FriendlyCounter.java b/akka-docs/java/code/akka/docs/transactor/FriendlyCounter.java index d70c653063..7ef31c5bea 100644 --- a/akka-docs/java/code/akka/docs/transactor/FriendlyCounter.java +++ b/akka-docs/java/code/akka/docs/transactor/FriendlyCounter.java @@ -8,10 +8,11 @@ package akka.docs.transactor; import akka.actor.*; import akka.transactor.*; import java.util.Set; -import scala.concurrent.stm.*; +import scala.concurrent.stm.Ref; +import static scala.concurrent.stm.JavaAPI.*; public class FriendlyCounter extends UntypedTransactor { - Ref count = Stm.ref(0); + Ref.View count = newRef(0); @Override public Set coordinate(Object message) { if (message instanceof Increment) { @@ -22,16 +23,15 @@ public class FriendlyCounter extends UntypedTransactor { return nobody(); } - public void atomically(InTxn txn, Object message) { + public void atomically(Object message) { if (message instanceof Increment) { - Integer newValue = count.get(txn) + 1; - count.set(newValue, txn); + increment(count, 1); } } @Override public boolean normally(Object message) { if ("GetCount".equals(message)) { - getSender().tell(count.single().get()); + getSender().tell(count.get()); return true; } else return false; } diff --git a/akka-docs/java/transactors.rst b/akka-docs/java/transactors.rst index f7471412a9..9dd69664b6 100644 --- a/akka-docs/java/transactors.rst +++ b/akka-docs/java/transactors.rst @@ -102,7 +102,7 @@ be sent. :language: java To enter the coordinated transaction use the atomic method of the coordinated -object, passing in an ``akka.transactor.Atomically`` object. +object, passing in a ``java.lang.Runnable``. .. includecode:: code/akka/docs/transactor/Coordinator.java#coordinated-atomic :language: java diff --git a/akka-transactor/src/main/scala/akka/transactor/Atomically.scala b/akka-transactor/src/main/scala/akka/transactor/Atomically.scala deleted file mode 100644 index 4995a6b8bd..0000000000 --- a/akka-transactor/src/main/scala/akka/transactor/Atomically.scala +++ /dev/null @@ -1,67 +0,0 @@ -/** - * Copyright (C) 2009-2011 Typesafe Inc. - */ - -package akka.transactor - -import scala.concurrent.stm._ - -/** - * Java API. - * - * For creating Java-friendly coordinated atomic blocks. - * - * @see [[akka.transactor.Coordinated]] - */ -trait Atomically { - def atomically(txn: InTxn): Unit -} - -/** - * Java API. - * - * For creating completion handlers. - */ -trait CompletionHandler { - def handle(status: Txn.Status): Unit -} - -/** - * Java API. - * - * To ease some of the pain of using Scala STM from Java until - * the proper Java API is created. - */ -object Stm { - /** - * Create an STM Ref with an initial value. - */ - def ref[A](initialValue: A): Ref[A] = Ref(initialValue) - - /** - * Add a CompletionHandler to run after the current transaction - * has committed. - */ - def afterCommit(handler: CompletionHandler): Unit = { - val txn = Txn.findCurrent - if (txn.isDefined) Txn.afterCommit(status ⇒ handler.handle(status))(txn.get) - } - - /** - * Add a CompletionHandler to run after the current transaction - * has rolled back. - */ - def afterRollback(handler: CompletionHandler): Unit = { - val txn = Txn.findCurrent - if (txn.isDefined) Txn.afterRollback(status ⇒ handler.handle(status))(txn.get) - } - - /** - * Add a CompletionHandler to run after the current transaction - * has committed or rolled back. - */ - def afterCompletion(handler: CompletionHandler): Unit = { - val txn = Txn.findCurrent - if (txn.isDefined) Txn.afterCompletion(status ⇒ handler.handle(status))(txn.get) - } -} diff --git a/akka-transactor/src/main/scala/akka/transactor/Coordinated.scala b/akka-transactor/src/main/scala/akka/transactor/Coordinated.scala index f9ef8538be..a7c709b9fe 100644 --- a/akka-transactor/src/main/scala/akka/transactor/Coordinated.scala +++ b/akka-transactor/src/main/scala/akka/transactor/Coordinated.scala @@ -6,7 +6,8 @@ package akka.transactor import akka.AkkaException import akka.util.Timeout -import scala.concurrent.stm._ +import scala.concurrent.stm.{ CommitBarrier, InTxn } +import java.util.concurrent.Callable /** * Akka-specific exception for coordinated transactions. @@ -125,7 +126,7 @@ class Coordinated(val message: Any, member: CommitBarrier.Member) { * * @throws CoordinatedTransactionException if the coordinated transaction fails. */ - def atomic[T](body: InTxn ⇒ T): T = { + def atomic[A](body: InTxn ⇒ A): A = { member.atomic(body) match { case Right(result) ⇒ result case Left(CommitBarrier.MemberUncaughtExceptionCause(x)) ⇒ @@ -136,13 +137,22 @@ class Coordinated(val message: Any, member: CommitBarrier.Member) { } /** - * Java API: coordinated atomic method that accepts an [[akka.transactor.Atomically]]. + * Java API: coordinated atomic method that accepts a `java.lang.Runnable`. * Delimits the coordinated transaction. The transaction will wait for all other transactions * in this coordination before committing. The timeout is specified when creating the Coordinated. * * @throws CoordinatedTransactionException if the coordinated transaction fails. */ - def atomic(atomically: Atomically): Unit = atomic(txn ⇒ atomically.atomically(txn)) + def atomic(runnable: Runnable): Unit = atomic { _ ⇒ runnable.run } + + /** + * Java API: coordinated atomic method that accepts a `java.util.concurrent.Callable`. + * Delimits the coordinated transaction. The transaction will wait for all other transactions + * in this coordination before committing. The timeout is specified when creating the Coordinated. + * + * @throws CoordinatedTransactionException if the coordinated transaction fails. + */ + def atomic[A](callable: Callable[A]): A = atomic { _ ⇒ callable.call } /** * An empty coordinated atomic block. Can be used to complete the number of members involved diff --git a/akka-transactor/src/main/scala/akka/transactor/UntypedTransactor.scala b/akka-transactor/src/main/scala/akka/transactor/UntypedTransactor.scala index 9a37f81915..59dc8f049d 100644 --- a/akka-transactor/src/main/scala/akka/transactor/UntypedTransactor.scala +++ b/akka-transactor/src/main/scala/akka/transactor/UntypedTransactor.scala @@ -25,7 +25,7 @@ abstract class UntypedTransactor extends UntypedActor { sendTo.actor.tell(coordinated(sendTo.message.getOrElse(message))) } before(message) - coordinated.atomic { txn ⇒ atomically(txn, message) } + coordinated.atomic { txn ⇒ atomically(message) } after(message) } case message ⇒ { @@ -84,7 +84,7 @@ abstract class UntypedTransactor extends UntypedActor { * The Receive block to run inside the coordinated transaction. */ @throws(classOf[Exception]) - def atomically(txn: InTxn, message: Any) {} + def atomically(message: Any) /** * A Receive block that runs after the coordinated transaction. diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedCounter.java b/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedCounter.java index 694a675d8e..7c92930e02 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedCounter.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedCounter.java @@ -7,24 +7,20 @@ package akka.transactor; import akka.actor.ActorRef; import akka.actor.Actors; import akka.actor.UntypedActor; -import scala.concurrent.stm.*; +import static scala.concurrent.stm.JavaAPI.*; +import scala.concurrent.stm.Ref; import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; public class UntypedCoordinatedCounter extends UntypedActor { private String name; - private Ref count = Stm.ref(0); + private Ref.View count = newRef(0); public UntypedCoordinatedCounter(String name) { this.name = name; } - private void increment(InTxn txn) { - Integer newValue = count.get(txn) + 1; - count.set(newValue, txn); - } - public void onReceive(Object incoming) throws Exception { if (incoming instanceof Coordinated) { Coordinated coordinated = (Coordinated) incoming; @@ -33,8 +29,8 @@ public class UntypedCoordinatedCounter extends UntypedActor { Increment increment = (Increment) message; List friends = increment.getFriends(); final CountDownLatch latch = increment.getLatch(); - final CompletionHandler countDown = new CompletionHandler() { - public void handle(Txn.Status status) { + final Runnable countDown = new Runnable() { + public void run() { latch.countDown(); } }; @@ -42,15 +38,16 @@ public class UntypedCoordinatedCounter extends UntypedActor { Increment coordMessage = new Increment(friends.subList(1, friends.size()), latch); friends.get(0).tell(coordinated.coordinate(coordMessage)); } - coordinated.atomic(new Atomically() { - public void atomically(InTxn txn) { - increment(txn); - Stm.afterCompletion(countDown); + coordinated.atomic(new Runnable() { + public void run() { + increment(count, 1); + afterRollback(countDown); + afterCommit(countDown); } }); } } else if ("GetCount".equals(incoming)) { - getSender().tell(count.single().get()); + getSender().tell(count.get()); } } } diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedCounter.java b/akka-transactor/src/test/java/akka/transactor/UntypedCounter.java index f03f74b10f..392bfbca42 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedCounter.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedCounter.java @@ -7,7 +7,8 @@ package akka.transactor; import akka.actor.ActorRef; import akka.transactor.UntypedTransactor; import akka.transactor.SendTo; -import scala.concurrent.stm.*; +import static scala.concurrent.stm.JavaAPI.*; +import scala.concurrent.stm.Ref; import java.util.List; import java.util.Set; import java.util.concurrent.CountDownLatch; @@ -15,17 +16,12 @@ import java.util.concurrent.TimeUnit; public class UntypedCounter extends UntypedTransactor { private String name; - private Ref count = Stm.ref(0); + private Ref.View count = newRef(0); public UntypedCounter(String name) { this.name = name; } - private void increment(InTxn txn) { - Integer newValue = count.get(txn) + 1; - count.set(newValue, txn); - } - @Override public Set coordinate(Object message) { if (message instanceof Increment) { Increment increment = (Increment) message; @@ -41,22 +37,23 @@ public class UntypedCounter extends UntypedTransactor { } } - public void atomically(InTxn txn, Object message) { + public void atomically(Object message) { if (message instanceof Increment) { - increment(txn); + increment(count, 1); final Increment increment = (Increment) message; - CompletionHandler countDown = new CompletionHandler() { - public void handle(Txn.Status status) { + Runnable countDown = new Runnable() { + public void run() { increment.getLatch().countDown(); } }; - Stm.afterCompletion(countDown); + afterRollback(countDown); + afterCommit(countDown); } } @Override public boolean normally(Object message) { if ("GetCount".equals(message)) { - getSender().tell(count.single().get()); + getSender().tell(count.get()); return true; } else return false; } diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedFailer.java b/akka-transactor/src/test/java/akka/transactor/UntypedFailer.java index 1f9e6ff41c..8ead9ae2ea 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedFailer.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedFailer.java @@ -7,7 +7,7 @@ package akka.transactor; import scala.concurrent.stm.InTxn; public class UntypedFailer extends UntypedTransactor { - public void atomically(InTxn txn, Object message) throws Exception { + public void atomically(Object message) throws Exception { throw new ExpectedFailureException(); } } diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 091346de34..626ede1834 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -450,7 +450,7 @@ object Dependency { val Netty = "3.2.5.Final" val Protobuf = "2.4.1" val Rabbit = "2.3.1" - val ScalaStm = "0.4" + val ScalaStm = "0.5.0-SNAPSHOT" val Scalatest = "1.6.1" val Slf4j = "1.6.4" val Spring = "3.0.5.RELEASE" From 20587654852144aadc824ea25cbd202081c6e9db Mon Sep 17 00:00:00 2001 From: Peter Vlugter Date: Thu, 19 Jan 2012 11:09:03 +1300 Subject: [PATCH 033/152] Add basic java api for scala stm to transactor module Note: this commit will be reverted once a java api is published for scala stm. Adding to transactor module for M3 release. --- .../scala/scala/concurrent/stm/JavaAPI.scala | 112 ++++++++++++ .../scala/concurrent/stm/JavaAPITests.java | 161 ++++++++++++++++++ .../scala/concurrent/stm/TestException.java | 9 + .../scala/concurrent/stm/JavaAPISuite.scala | 7 + project/AkkaBuild.scala | 2 +- 5 files changed, 290 insertions(+), 1 deletion(-) create mode 100644 akka-transactor/src/main/scala/scala/concurrent/stm/JavaAPI.scala create mode 100644 akka-transactor/src/test/java/scala/concurrent/stm/JavaAPITests.java create mode 100644 akka-transactor/src/test/java/scala/concurrent/stm/TestException.java create mode 100644 akka-transactor/src/test/scala/scala/concurrent/stm/JavaAPISuite.scala diff --git a/akka-transactor/src/main/scala/scala/concurrent/stm/JavaAPI.scala b/akka-transactor/src/main/scala/scala/concurrent/stm/JavaAPI.scala new file mode 100644 index 0000000000..964664fe55 --- /dev/null +++ b/akka-transactor/src/main/scala/scala/concurrent/stm/JavaAPI.scala @@ -0,0 +1,112 @@ +/* scala-stm - (c) 2009-2011, Stanford University, PPL */ + +package scala.concurrent.stm + +import java.util.concurrent.Callable +import scala.runtime.AbstractFunction1 + +/** + * Java-friendly API. + */ +object JavaAPI { + + /** + * Create a Ref with an initial value. Return a `Ref.View`, which does not + * require implicit transactions. + * @param initialValue the initial value for the newly created `Ref.View` + * @return a new `Ref.View` + */ + def newRef[A](initialValue: A): Ref.View[A] = Ref(initialValue).single + + /** + * Create an empty TMap. Return a `TMap.View`, which does not require + * implicit transactions. + * @return a new, empty `TMap.View` + */ + def newTMap[A, B](): TMap.View[A, B] = TMap.empty[A, B].single + + /** + * Create an empty TSet. Return a `TSet.View`, which does not require + * implicit transactions. + * @return a new, empty `TSet.View` + */ + def newTSet[A](): TSet.View[A] = TSet.empty[A].single + + /** + * Create a TArray containing `length` elements. Return a `TArray.View`, + * which does not require implicit transactions. + * @param length the length of the `TArray.View` to be created + * @return a new `TArray.View` containing `length` elements (initially null) + */ + def newTArray[A <: AnyRef](length: Int): TArray.View[A] = TArray.ofDim[A](length)(ClassManifest.classType(AnyRef.getClass)).single + + /** + * Atomic block that takes a `Runnable`. + * @param runnable the `Runnable` to run within a transaction + */ + def atomic(runnable: Runnable): Unit = scala.concurrent.stm.atomic { txn ⇒ runnable.run } + + /** + * Atomic block that takes a `Callable`. + * @param callable the `Callable` to run within a transaction + * @return the value returned by the `Callable` + */ + def atomic[A](callable: Callable[A]): A = scala.concurrent.stm.atomic { txn ⇒ callable.call } + + /** + * Transform the value stored by `ref` by applying the function `f`. + * @param ref the `Ref.View` to be transformed + * @param f the function to be applied + */ + def transform[A](ref: Ref.View[A], f: AbstractFunction1[A, A]): Unit = ref.transform(f) + + /** + * Transform the value stored by `ref` by applying the function `f` and + * return the old value. + * @param ref the `Ref.View` to be transformed + * @param f the function to be applied + * @return the old value of `ref` + */ + def getAndTransform[A](ref: Ref.View[A], f: AbstractFunction1[A, A]): A = ref.getAndTransform(f) + + /** + * Transform the value stored by `ref` by applying the function `f` and + * return the new value. + * @param ref the `Ref.View` to be transformed + * @param f the function to be applied + * @return the new value of `ref` + */ + def transformAndGet[A](ref: Ref.View[A], f: AbstractFunction1[A, A]): A = ref.transformAndGet(f) + + /** + * Increment the `java.lang.Integer` value of a `Ref.View`. + * @param ref the `Ref.View` to be incremented + * @param delta the amount to increment + */ + def increment(ref: Ref.View[java.lang.Integer], delta: Int): Unit = ref.transform { v ⇒ v.intValue + delta } + + /** + * Increment the `java.lang.Long` value of a `Ref.View`. + * @param ref the `Ref.View` to be incremented + * @param delta the amount to increment + */ + def increment(ref: Ref.View[java.lang.Long], delta: Long): Unit = ref.transform { v ⇒ v.longValue + delta } + + /** + * Add a task to run after the current transaction has committed. + * @param task the `Runnable` task to run after transaction commit + */ + def afterCommit(task: Runnable): Unit = { + val txn = Txn.findCurrent + if (txn.isDefined) Txn.afterCommit(status ⇒ task.run)(txn.get) + } + + /** + * Add a task to run after the current transaction has rolled back. + * @param task the `Runnable` task to run after transaction rollback + */ + def afterRollback(task: Runnable): Unit = { + val txn = Txn.findCurrent + if (txn.isDefined) Txn.afterRollback(status ⇒ task.run)(txn.get) + } +} diff --git a/akka-transactor/src/test/java/scala/concurrent/stm/JavaAPITests.java b/akka-transactor/src/test/java/scala/concurrent/stm/JavaAPITests.java new file mode 100644 index 0000000000..e2d0631590 --- /dev/null +++ b/akka-transactor/src/test/java/scala/concurrent/stm/JavaAPITests.java @@ -0,0 +1,161 @@ +/* scala-stm - (c) 2009-2011, Stanford University, PPL */ + +package scala.concurrent.stm; + +import static org.junit.Assert.*; +import org.junit.Test; + +import scala.concurrent.stm.Ref; +import static scala.concurrent.stm.JavaAPI.*; + +import scala.runtime.AbstractFunction1; +import java.util.concurrent.Callable; + +import static scala.collection.JavaConversions.*; +import java.util.Map; +import java.util.Set; +import java.util.List; + +public class JavaAPITests { + @Test + public void createIntegerRef() { + Ref.View ref = newRef(0); + int unboxed = ref.get(); + assertEquals(0, unboxed); + } + + @Test + public void atomicWithRunnable() { + final Ref.View ref = newRef(0); + atomic(new Runnable() { + public void run() { + ref.set(10); + } + }); + int value = ref.get(); + assertEquals(10, value); + } + + @Test + public void atomicWithCallable() { + final Ref.View ref = newRef(0); + int oldValue = atomic(new Callable() { + public Integer call() { + return ref.swap(10); + } + }); + assertEquals(0, oldValue); + int newValue = ref.get(); + assertEquals(10, newValue); + } + + @Test(expected = TestException.class) + public void failingTransaction() { + final Ref.View ref = newRef(0); + try { + atomic(new Runnable() { + public void run() { + ref.set(10); + throw new TestException(); + } + }); + } catch (TestException e) { + int value = ref.get(); + assertEquals(0, value); + throw e; + } + } + + @Test + public void transformInteger() { + Ref.View ref = newRef(0); + transform(ref, new AbstractFunction1() { + public Integer apply(Integer i) { + return i + 10; + } + }); + int value = ref.get(); + assertEquals(10, value); + } + + @Test + public void incrementInteger() { + Ref.View ref = newRef(0); + increment(ref, 10); + int value = ref.get(); + assertEquals(10, value); + } + + @Test + public void incrementLong() { + Ref.View ref = newRef(0L); + increment(ref, 10L); + long value = ref.get(); + assertEquals(10L, value); + } + + @Test + public void createAndUseTMap() { + TMap.View tmap = newTMap(); + Map map = mutableMapAsJavaMap(tmap); + map.put(1, "one"); + map.put(2, "two"); + assertEquals("one", map.get(1)); + assertEquals("two", map.get(2)); + assertTrue(map.containsKey(2)); + map.remove(2); + assertFalse(map.containsKey(2)); + } + + @Test(expected = TestException.class) + public void failingTMapTransaction() { + TMap.View tmap = newTMap(); + final Map map = mutableMapAsJavaMap(tmap); + try { + atomic(new Runnable() { + public void run() { + map.put(1, "one"); + map.put(2, "two"); + assertTrue(map.containsKey(1)); + assertTrue(map.containsKey(2)); + throw new TestException(); + } + }); + } catch (TestException e) { + assertFalse(map.containsKey(1)); + assertFalse(map.containsKey(2)); + throw e; + } + } + + @Test + public void createAndUseTSet() { + TSet.View tset = newTSet(); + Set set = mutableSetAsJavaSet(tset); + set.add("one"); + set.add("two"); + assertTrue(set.contains("one")); + assertTrue(set.contains("two")); + assertEquals(2, set.size()); + set.add("one"); + assertEquals(2, set.size()); + set.remove("two"); + assertFalse(set.contains("two")); + assertEquals(1, set.size()); + } + + @Test + public void createAndUseTArray() { + TArray.View tarray = newTArray(3); + List seq = mutableSeqAsJavaList(tarray); + assertEquals(null, seq.get(0)); + assertEquals(null, seq.get(1)); + assertEquals(null, seq.get(2)); + seq.set(0, "zero"); + seq.set(1, "one"); + seq.set(2, "two"); + assertEquals("zero", seq.get(0)); + assertEquals("one", seq.get(1)); + assertEquals("two", seq.get(2)); + } +} diff --git a/akka-transactor/src/test/java/scala/concurrent/stm/TestException.java b/akka-transactor/src/test/java/scala/concurrent/stm/TestException.java new file mode 100644 index 0000000000..cc810761d4 --- /dev/null +++ b/akka-transactor/src/test/java/scala/concurrent/stm/TestException.java @@ -0,0 +1,9 @@ +/* scala-stm - (c) 2009-2011, Stanford University, PPL */ + +package scala.concurrent.stm; + +public class TestException extends RuntimeException { + public TestException() { + super("Expected failure"); + } +} diff --git a/akka-transactor/src/test/scala/scala/concurrent/stm/JavaAPISuite.scala b/akka-transactor/src/test/scala/scala/concurrent/stm/JavaAPISuite.scala new file mode 100644 index 0000000000..3d0c48e90f --- /dev/null +++ b/akka-transactor/src/test/scala/scala/concurrent/stm/JavaAPISuite.scala @@ -0,0 +1,7 @@ +/* scala-stm - (c) 2009-2011, Stanford University, PPL */ + +package scala.concurrent.stm + +import org.scalatest.junit.JUnitWrapperSuite + +class JavaAPISuite extends JUnitWrapperSuite("scala.concurrent.stm.JavaAPITests", Thread.currentThread.getContextClassLoader) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 626ede1834..091346de34 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -450,7 +450,7 @@ object Dependency { val Netty = "3.2.5.Final" val Protobuf = "2.4.1" val Rabbit = "2.3.1" - val ScalaStm = "0.5.0-SNAPSHOT" + val ScalaStm = "0.4" val Scalatest = "1.6.1" val Slf4j = "1.6.4" val Spring = "3.0.5.RELEASE" From b9bbb0744a06a7529046fa10f41f743f03a6dbb4 Mon Sep 17 00:00:00 2001 From: Peter Vlugter Date: Wed, 18 Jan 2012 19:01:46 +1300 Subject: [PATCH 034/152] Add some migration guidance for stm --- .../project/migration-guide-1.3.x-2.0.x.rst | 280 +++++++++++++++++- 1 file changed, 266 insertions(+), 14 deletions(-) diff --git a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst index 353e0c0ddb..2749f0107c 100644 --- a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst +++ b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst @@ -6,7 +6,9 @@ .. sidebar:: Contents - .. contents:: :local: + .. contents:: + :local: + :depth: 3 Actors ====== @@ -77,8 +79,11 @@ Last task of the migration would be to create your own ``ActorSystem``. Unordered Collection of Migration Items ======================================= +Actors +------ + Creating and starting actors ----------------------------- +^^^^^^^^^^^^^^^^^^^^^^^^^^^^ Actors are created by passing in a ``Props`` instance into the actorOf factory method in a ``ActorRefProvider``, which is the ``ActorSystem`` or ``ActorContext``. @@ -111,7 +116,7 @@ Documentation: * :ref:`untyped-actors-java` Stopping actors ---------------- +^^^^^^^^^^^^^^^ ``ActorRef.stop()`` has been moved. Use ``ActorSystem`` or ``ActorContext`` to stop actors. @@ -144,7 +149,7 @@ Documentation: * :ref:`untyped-actors-java` Identifying Actors ------------------- +^^^^^^^^^^^^^^^^^^ In v1.3 actors have ``uuid`` and ``id`` field. In v2.0 each actor has a unique logical ``path``. @@ -167,7 +172,7 @@ Documentation: * :ref:`untyped-actors-java` Reply to messages ------------------ +^^^^^^^^^^^^^^^^^ ``self.channel`` has been replaced with unified reply mechanism using ``sender`` (Scala) or ``getSender()`` (Java). This works for both tell (!) and ask (?). @@ -189,7 +194,7 @@ Documentation: * :ref:`untyped-actors-java` ``ActorRef.ask()`` ------------------- +^^^^^^^^^^^^^^^^^^ The mechanism for collecting an actor’s reply in a :class:`Future` has been reworked for better location transparency: it uses an actor under the hood. @@ -206,7 +211,7 @@ Documentation: * :ref:`untyped-actors-java` ActorPool ---------- +^^^^^^^^^ The ActorPool has been replaced by dynamically resizable routers. @@ -216,7 +221,7 @@ Documentation: * :ref:`routing-java` ``UntypedActor.getContext()`` (Java API only) ---------------------------------------------- +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ ``getContext()`` in the Java API for UntypedActor is renamed to ``getSelf()``. @@ -234,7 +239,7 @@ Documentation: * :ref:`untyped-actors-java` Logging -------- +^^^^^^^ EventHandler API has been replaced by LoggingAdapter, which publish log messages to the event bus. You can still plugin your own actor as event listener with the @@ -267,7 +272,7 @@ Documentation: * :ref:`event-bus-java` Supervision ------------ +^^^^^^^^^^^ Akka v2.0 implements parental supervision. Actors can only be created by other actors — where the top-level actor is provided by the library — and each created actor is supervised by its parent. @@ -343,7 +348,7 @@ Documentation: * :ref:`untyped-actors-java` Spawn ------ +^^^^^ ``spawn`` has been removed and can be implemented like this, if needed. Be careful to not access any shared mutable state closed over by the body. @@ -359,7 +364,7 @@ Documentation: * :ref:`jmm` HotSwap -------- +^^^^^^^ In v2.0 ``become`` and ``unbecome`` metods are located in ``ActorContext``, i.e. ``context.become`` and ``context.unbecome``. @@ -370,15 +375,262 @@ in the actor receiving the message. * :ref:`actors-scala` * :ref:`untyped-actors-java` +STM +--- + +In Akka v2.0 `ScalaSTM`_ is used rather than Multiverse. + +.. _ScalaSTM: http://nbronson.github.com/scala-stm/ + +Agent and Transactor have been ported to ScalaSTM. The API's for Agent and +Transactor are basically the same, other than integration with ScalaSTM. See: + + * :ref:`agents-scala` + * :ref:`agents-java` + * :ref:`transactors-scala` + * :ref:`transactors-java` + +Imports +^^^^^^^ + +Scala +~~~~~ + +To use ScalaSTM the import from Scala is:: + + import scala.concurrent.stm._ + +Java +~~~~ + +For Java there is a special JavaAPI helper object that can be statically +imported, along with any other imports that might be needed:: + + import scala.concurrent.stm.Ref; + import static scala.concurrent.stm.JavaAPI.*; + +Transactions +^^^^^^^^^^^^ + +Scala +~~~~~ + +Both v1.3 and v2.0 provide an ``atomic`` block, however, the ScalaSTM ``atomic`` +is a function from ``InTxn`` to return type. + +v1.3:: + + atomic { + // do something in transaction + } + +v2.0:: + + atomic { implicit txn => + // do something in transaction + } + +Note that in ScalaSTM the ``InTxn`` in the atomic function is usually marked as +implicit as transactional references require an implicit ``InTxn`` on all +methods. That is, the transaction is statically required and it is a +compile-time warning to use a reference without a transaction. There is also a +``Ref.View`` for operations without requiring an ``InTxn`` statically. See below +for more information. + +Java +~~~~ + +In the ScalaSTM JavaAPI helpers there are atomic methods which accept +``java.lang.Runnable`` and ``java.util.concurrent.Callable``. + +v1.3:: + + new Atomic() { + public Object atomically() { + // in transaction + return null; + } + }.execute(); + + SomeObject result = new Atomic() { + public SomeObject atomically() { + // in transaction + return ...; + } + }.execute(); + +v2.0:: + + import static scala.concurrent.stm.JavaAPI.*; + import java.util.concurrent.Callable; + + atomic(new Runnable() { + public void run() { + // in transaction + } + }); + + SomeObject result = atomic(new Callable() { + public SomeObject call() { + // in transaction + return ...; + } + }); + +Ref +^^^ + +Scala +~~~~~ + +Other than the import, creating a Ref is basically identical between Akka STM in +v1.3 and ScalaSTM used in v2.0. + +v1.3:: + + val ref = Ref(0) + +v2.0:: + + val ref = Ref(0) + +The API for Ref is similar. For example: + +v1.3:: + + ref.get // get current value + ref() // same as get + + ref.set(1) // set to new value, return old value + ref() = 1 // same as set + ref.swap(2) // same as set + + ref alter { _ + 1 } // apply a function, return new value + +v2.0:: + + ref.get // get current value + ref() // same as get + + ref.set(1) // set to new value, return nothing + ref() = 1 // same as set + ref.swap(2) // set and return old value + + ref transform { _ + 1 } // apply function, return nothing + + ref transformIfDefined { case 1 => 2 } // apply partial function if defined + +Ref.View +^^^^^^^^ + +In v1.3 using a ``Ref`` method outside of a transaction would automatically +create a single-operation transaction. In v2.0 (in ScalaSTM) there is a +``Ref.View`` which provides methods without requiring a current +transaction. + +Scala +~~~~~ + +The ``Ref.View`` can be accessed with the ``single`` method:: + + ref.single() // returns current value + ref.single() = 1 // set new value + + // with atomic this would be: + + atomic { implicit t => ref() } + atomic { implicit t => ref() = 1 } + +Java +~~~~ + +As ``Ref.View`` in ScalaSTM does not require implicit transactions, this is more +easily used from Java. ``Ref`` could be used, but requires explicit threading of +transactions. There are helper methods in ``JavaAPI`` for creating ``Ref.View`` +references. + +v1.3:: + + Ref ref = new Ref(0); + +v2.0:: + + Ref.View ref = newRef(0); + +The ``set`` and ``get`` methods work the same way for both versions. + +v1.3:: + + ref.get(); // get current value + ref.set(1); // set new value + +v2.0:: + + ref.get(); // get current value + ref.set(1); // set new value + +There are also ``transform``, ``getAndTransform``, and ``transformAndGet`` +methods in ``JavaAPI`` which accept ``scala.runtime.AbstractFunction1``. + +There are ``increment`` helper methods for ``Ref.View`` and +``Ref.View`` references. + +Transaction lifecycle callbacks +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +Scala +~~~~~ + +It is also possible to hook into the transaction lifecycle in ScalaSTM. See the +ScalaSTM documentation for the full range of possibilities. + +v1.3:: + + atomic { + deferred { + // executes when transaction commits + } + compensating { + // executes when transaction aborts + } + } + +v2.0:: + + atomic { implicit txn => + txn.afterCommit { txnStatus => + // executes when transaction commits + } + txn.afterRollback { txnStatus => + // executes when transaction rolls back + } + } + +Java +~~~~ + +Rather than using the ``deferred`` and ``compensating`` methods in +``akka.stm.StmUtils``, use the ``afterCommit`` and ``afterRollback`` methods in +``scala.concurrent.stm.JavaAPI``, which behave in the same way and accept +``Runnable``. + +Transactional Datastructures +^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +In ScalaSTM see ``TMap``, ``TSet``, and ``TArray`` for transactional +datastructures. There are helper methods for creating these in +``JavaAPI``. These datastructure implement the ``scala.collection`` interfaces +and can also be used from Java with Scala's ``JavaConversions``. + + More to be written ------------------ * Futures * Dispatchers -* STM * TypedActors * Routing * Remoting * Scheduler * Configuration -* ...? \ No newline at end of file +* ...? From d9a8c4e7e68476a99d3438ff031cf8f1c8935241 Mon Sep 17 00:00:00 2001 From: Ivan Porto Carrero Date: Thu, 19 Jan 2012 00:26:52 +0100 Subject: [PATCH 035/152] Changes the intialization logic for the actor so that zeromq options are preserved --- .../additional/third-party-integrations.rst | 2 +- .../akka/zeromq/ConcurrentSocketActor.scala | 125 +++++++++++------- .../src/main/scala/akka/zeromq/Context.scala | 20 --- .../main/scala/akka/zeromq/Deserializer.scala | 2 +- .../src/main/scala/akka/zeromq/Requests.scala | 35 ++++- .../main/scala/akka/zeromq/SocketType.scala | 14 -- .../scala/akka/zeromq/ZeroMQExtension.scala | 34 ++--- .../zeromq/ConcurrentSocketActorSpec.scala | 12 +- 8 files changed, 132 insertions(+), 112 deletions(-) delete mode 100644 akka-zeromq/src/main/scala/akka/zeromq/Context.scala delete mode 100644 akka-zeromq/src/main/scala/akka/zeromq/SocketType.scala diff --git a/akka-docs/additional/third-party-integrations.rst b/akka-docs/additional/third-party-integrations.rst index 446c8436b3..93b3b36d2e 100644 --- a/akka-docs/additional/third-party-integrations.rst +++ b/akka-docs/additional/third-party-integrations.rst @@ -13,5 +13,5 @@ Scalatra Scalatra has Akka integration. -Read more here: ``_ +Read more here: ``_ diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala index ca7cc08e89..07a7b5f888 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala @@ -8,17 +8,22 @@ import org.zeromq.{ ZMQ ⇒ JZMQ } import akka.actor._ import akka.dispatch.{ Promise, Future } import akka.event.Logging +import akka.util.duration._ private[zeromq] sealed trait PollLifeCycle private[zeromq] case object NoResults extends PollLifeCycle private[zeromq] case object Results extends PollLifeCycle private[zeromq] case object Closing extends PollLifeCycle -private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Actor { +private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends Actor { private val noBytes = Array[Byte]() - private val socket: Socket = params.context.socket(params.socketType) - private val poller: Poller = params.context.poller + private val zmqContext = { + params find (_.isInstanceOf[Context]) map (_.asInstanceOf[Context]) getOrElse new Context(1) + } + private lazy val deserializer = deserializerFromParams + private lazy val socket: Socket = socketFromParams + private lazy val poller: Poller = zmqContext.poller private val log = Logging(context.system, this) private case object Poll @@ -55,45 +60,26 @@ private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Ac } private def handleSocketOption: Receive = { - case Linger(value) ⇒ socket.setLinger(value) - case ReconnectIVL(value) ⇒ socket.setReconnectIVL(value) - case Backlog(value) ⇒ socket.setBacklog(value) - case ReconnectIVLMax(value) ⇒ socket.setReconnectIVLMax(value) - case MaxMsgSize(value) ⇒ socket.setMaxMsgSize(value) - case SndHWM(value) ⇒ socket.setSndHWM(value) - case RcvHWM(value) ⇒ socket.setRcvHWM(value) - case HWM(value) ⇒ socket.setHWM(value) - case Swap(value) ⇒ socket.setSwap(value) - case Affinity(value) ⇒ socket.setAffinity(value) - case Identity(value) ⇒ socket.setIdentity(value) - case Rate(value) ⇒ socket.setRate(value) - case RecoveryInterval(value) ⇒ socket.setRecoveryInterval(value) - case MulticastLoop(value) ⇒ socket.setMulticastLoop(value) - case MulticastHops(value) ⇒ socket.setMulticastHops(value) - case ReceiveTimeOut(value) ⇒ socket.setReceiveTimeOut(value) - case SendTimeOut(value) ⇒ socket.setSendTimeOut(value) - case SendBufferSize(value) ⇒ socket.setSendBufferSize(value) - case ReceiveBufferSize(value) ⇒ socket.setReceiveBufferSize(value) - case Linger ⇒ sender ! socket.getLinger - case ReconnectIVL ⇒ sender ! socket.getReconnectIVL - case Backlog ⇒ sender ! socket.getBacklog - case ReconnectIVLMax ⇒ sender ! socket.getReconnectIVLMax - case MaxMsgSize ⇒ sender ! socket.getMaxMsgSize - case SndHWM ⇒ sender ! socket.getSndHWM - case RcvHWM ⇒ sender ! socket.getRcvHWM - case Swap ⇒ sender ! socket.getSwap - case Affinity ⇒ sender ! socket.getAffinity - case Identity ⇒ sender ! socket.getIdentity - case Rate ⇒ sender ! socket.getRate - case RecoveryInterval ⇒ sender ! socket.getRecoveryInterval - case MulticastLoop ⇒ sender ! socket.hasMulticastLoop - case MulticastHops ⇒ sender ! socket.getMulticastHops - case ReceiveTimeOut ⇒ sender ! socket.getReceiveTimeOut - case SendTimeOut ⇒ sender ! socket.getSendTimeOut - case SendBufferSize ⇒ sender ! socket.getSendBufferSize - case ReceiveBufferSize ⇒ sender ! socket.getReceiveBufferSize - case ReceiveMore ⇒ sender ! socket.hasReceiveMore - case FileDescriptor ⇒ sender ! socket.getFD + case Linger ⇒ sender ! socket.getLinger + case ReconnectIVL ⇒ sender ! socket.getReconnectIVL + case Backlog ⇒ sender ! socket.getBacklog + case ReconnectIVLMax ⇒ sender ! socket.getReconnectIVLMax + case MaxMsgSize ⇒ sender ! socket.getMaxMsgSize + case SndHWM ⇒ sender ! socket.getSndHWM + case RcvHWM ⇒ sender ! socket.getRcvHWM + case Swap ⇒ sender ! socket.getSwap + case Affinity ⇒ sender ! socket.getAffinity + case Identity ⇒ sender ! socket.getIdentity + case Rate ⇒ sender ! socket.getRate + case RecoveryInterval ⇒ sender ! socket.getRecoveryInterval + case MulticastLoop ⇒ sender ! socket.hasMulticastLoop + case MulticastHops ⇒ sender ! socket.getMulticastHops + case ReceiveTimeOut ⇒ sender ! socket.getReceiveTimeOut + case SendTimeOut ⇒ sender ! socket.getSendTimeOut + case SendBufferSize ⇒ sender ! socket.getSendBufferSize + case ReceiveBufferSize ⇒ sender ! socket.getReceiveBufferSize + case ReceiveMore ⇒ sender ! socket.hasReceiveMore + case FileDescriptor ⇒ sender ! socket.getFD } private def internalMessage: Receive = { @@ -104,7 +90,7 @@ private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Ac case ReceiveFrames ⇒ { receiveFrames() match { case Seq() ⇒ - case frames ⇒ notifyListener(params.deserializer(frames)) + case frames ⇒ notifyListener(deserializer(frames)) } self ! Poll } @@ -118,9 +104,46 @@ private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Ac override def receive: Receive = handleConnectionMessages orElse handleSocketOption orElse internalMessage override def preStart { + setupSocket() poller.register(socket, Poller.POLLIN) } + private def socketFromParams() = { + require(ZeroMQExtension.check[SocketType.ZMQSocketType](params), "A socket type is required") + (params + find (_.isInstanceOf[SocketType.ZMQSocketType]) + map (t ⇒ zmqContext.socket(t.asInstanceOf[SocketType.ZMQSocketType])) get) + } + + private def deserializerFromParams = { + params find (_.isInstanceOf[Deserializer]) map (_.asInstanceOf[Deserializer]) getOrElse new ZMQMessageDeserializer + } + + private def setupSocket() = { + params foreach { + case Linger(value) ⇒ socket.setLinger(value) + case ReconnectIVL(value) ⇒ socket.setReconnectIVL(value) + case Backlog(value) ⇒ socket.setBacklog(value) + case ReconnectIVLMax(value) ⇒ socket.setReconnectIVLMax(value) + case MaxMsgSize(value) ⇒ socket.setMaxMsgSize(value) + case SndHWM(value) ⇒ socket.setSndHWM(value) + case RcvHWM(value) ⇒ socket.setRcvHWM(value) + case HWM(value) ⇒ socket.setHWM(value) + case Swap(value) ⇒ socket.setSwap(value) + case Affinity(value) ⇒ socket.setAffinity(value) + case Identity(value) ⇒ socket.setIdentity(value) + case Rate(value) ⇒ socket.setRate(value) + case RecoveryInterval(value) ⇒ socket.setRecoveryInterval(value) + case MulticastLoop(value) ⇒ socket.setMulticastLoop(value) + case MulticastHops(value) ⇒ socket.setMulticastHops(value) + case ReceiveTimeOut(value) ⇒ socket.setReceiveTimeOut(value) + case SendTimeOut(value) ⇒ socket.setSendTimeOut(value) + case SendBufferSize(value) ⇒ socket.setSendBufferSize(value) + case ReceiveBufferSize(value) ⇒ socket.setReceiveBufferSize(value) + case _ ⇒ + } + } + override def postStop { try { poller.unregister(socket) @@ -146,14 +169,22 @@ private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Ac if (currentPoll.isEmpty) currentPoll = newEventLoop } - private val eventLoopDispatcher = { - params.pollDispatcher.map(d ⇒ context.system.dispatchers.lookup(d)) getOrElse context.system.dispatcher + private lazy val eventLoopDispatcher = { + val fromConfig = params.find(_.isInstanceOf[PollDispatcher]) map { + option ⇒ context.system.dispatchers.lookup(option.asInstanceOf[PollDispatcher].name) + } + fromConfig getOrElse context.system.dispatcher + } + + private lazy val pollTimeout = { + val fromConfig = params find (_.isInstanceOf[PollTimeoutDuration]) map (_.asInstanceOf[PollTimeoutDuration].duration) + fromConfig getOrElse 100.millis } private def newEventLoop: Option[Promise[PollLifeCycle]] = { implicit val executor = eventLoopDispatcher Some((Future { - if (poller.poll(params.pollTimeoutDuration.toMicros) > 0 && poller.pollin(0)) Results else NoResults + if (poller.poll(pollTimeout.toMicros) > 0 && poller.pollin(0)) Results else NoResults }).asInstanceOf[Promise[PollLifeCycle]] onSuccess { case Results ⇒ self ! ReceiveFrames case NoResults ⇒ self ! Poll @@ -184,7 +215,7 @@ private[zeromq] class ConcurrentSocketActor(params: SocketParameters) extends Ac } private def notifyListener(message: Any) { - params.listener.foreach { listener ⇒ + params find (_.isInstanceOf[Listener]) map (_.asInstanceOf[Listener].listener) foreach { listener ⇒ if (listener.isTerminated) context stop self else diff --git a/akka-zeromq/src/main/scala/akka/zeromq/Context.scala b/akka-zeromq/src/main/scala/akka/zeromq/Context.scala deleted file mode 100644 index 7d768e1492..0000000000 --- a/akka-zeromq/src/main/scala/akka/zeromq/Context.scala +++ /dev/null @@ -1,20 +0,0 @@ -/** - * Copyright (C) 2009-2011 Typesafe Inc. - */ -package akka.zeromq - -import org.zeromq.{ ZMQ ⇒ JZMQ } -import akka.zeromq.SocketType._ - -class Context(numIoThreads: Int) { - private val context = JZMQ.context(numIoThreads) - def socket(socketType: SocketType) = { - context.socket(socketType.id) - } - def poller = { - context.poller - } - def term = { - context.term - } -} diff --git a/akka-zeromq/src/main/scala/akka/zeromq/Deserializer.scala b/akka-zeromq/src/main/scala/akka/zeromq/Deserializer.scala index 6430a5a9c6..daf75f7c4b 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/Deserializer.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/Deserializer.scala @@ -6,7 +6,7 @@ package akka.zeromq case class Frame(payload: Seq[Byte]) object Frame { def apply(s: String): Frame = Frame(s.getBytes) } -trait Deserializer { +trait Deserializer extends SocketOption { def apply(frames: Seq[Frame]): Any } diff --git a/akka-zeromq/src/main/scala/akka/zeromq/Requests.scala b/akka-zeromq/src/main/scala/akka/zeromq/Requests.scala index 3cb5388069..0b1a2dad90 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/Requests.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/Requests.scala @@ -4,12 +4,45 @@ package akka.zeromq import com.google.protobuf.Message +import org.zeromq.{ ZMQ ⇒ JZMQ } +import akka.actor.ActorRef +import akka.util.duration._ +import akka.util.Duration sealed trait Request -sealed trait SocketOption extends Request +trait SocketOption extends Request sealed trait SocketOptionQuery extends Request case class Connect(endpoint: String) extends Request + +object Context { + def apply(numIoThreads: Int = 1) = new Context(numIoThreads) +} +class Context(numIoThreads: Int) extends SocketOption { + private val context = JZMQ.context(numIoThreads) + def socket(socketType: SocketType.ZMQSocketType) = { + context.socket(socketType.id) + } + def poller = { + context.poller + } + def term = { + context.term + } +} + +object SocketType { + abstract class ZMQSocketType(val id: Int) extends SocketOption + object Pub extends ZMQSocketType(JZMQ.PUB) + object Sub extends ZMQSocketType(JZMQ.SUB) + object Dealer extends ZMQSocketType(JZMQ.DEALER) + object Router extends ZMQSocketType(JZMQ.ROUTER) +} + +case class Listener(listener: ActorRef) extends SocketOption +case class PollDispatcher(name: String) extends SocketOption +case class PollTimeoutDuration(duration: Duration = 100 millis) extends SocketOption + case class Bind(endpoint: String) extends Request private[zeromq] case object Close extends Request diff --git a/akka-zeromq/src/main/scala/akka/zeromq/SocketType.scala b/akka-zeromq/src/main/scala/akka/zeromq/SocketType.scala deleted file mode 100644 index aba0c1608a..0000000000 --- a/akka-zeromq/src/main/scala/akka/zeromq/SocketType.scala +++ /dev/null @@ -1,14 +0,0 @@ -/** - * Copyright (C) 2009-2011 Typesafe Inc. - */ -package akka.zeromq - -import org.zeromq.{ ZMQ ⇒ JZMQ } - -object SocketType extends Enumeration { - type SocketType = Value - val Pub = Value(JZMQ.PUB) - val Sub = Value(JZMQ.SUB) - val Dealer = Value(JZMQ.DEALER) - val Router = Value(JZMQ.ROUTER) -} diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala index e44bc00d06..685e3106bd 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala @@ -5,18 +5,10 @@ package akka.zeromq import akka.util.Duration import akka.util.duration._ -import akka.zeromq.SocketType._ import org.zeromq.{ ZMQ ⇒ JZMQ } import akka.actor._ import akka.dispatch.{ Dispatcher, Await } - -case class SocketParameters( - socketType: SocketType, - context: Context, - listener: Option[ActorRef] = None, - pollDispatcher: Option[String] = None, - deserializer: Deserializer = new ZMQMessageDeserializer, - pollTimeoutDuration: Duration = 100 millis) +import collection.mutable.ListBuffer case class ZeroMQVersion(major: Int, minor: Int, patch: Int) { override def toString = "%d.%d.%d".format(major, minor, patch) @@ -28,6 +20,12 @@ object ZeroMQExtension extends ExtensionId[ZeroMQExtension] with ExtensionIdProv private val minVersionString = "2.1.0" private val minVersion = JZMQ.makeVersion(2, 1, 0) + + private[zeromq] def check[TOption <: SocketOption: Manifest](parameters: Seq[SocketOption]) = { + parameters exists { p ⇒ + ClassManifest.singleType(p) <:< manifest[TOption] + } + } } class ZeroMQExtension(system: ActorSystem) extends Extension { @@ -35,23 +33,15 @@ class ZeroMQExtension(system: ActorSystem) extends Extension { ZeroMQVersion(JZMQ.getMajorVersion, JZMQ.getMinorVersion, JZMQ.getPatchVersion) } - lazy val DefaultContext = newContext() - - def newContext(numIoThreads: Int = 1) = { + def newSocketProps(socketParameters: SocketOption*): Props = { verifyZeroMQVersion - new Context(numIoThreads) + require(ZeroMQExtension.check[SocketType.ZMQSocketType](socketParameters), "A socket type is required") + Props(new ConcurrentSocketActor(socketParameters)).withDispatcher("akka.zeromq.socket-dispatcher") } - def newSocket(socketType: SocketType, - listener: Option[ActorRef] = None, - context: Context = DefaultContext, // For most applications you want to use the default context - deserializer: Deserializer = new ZMQMessageDeserializer, - pollDispatcher: Option[String] = None, - pollTimeoutDuration: Duration = 500 millis): ActorRef = { - verifyZeroMQVersion - val params = SocketParameters(socketType, context, listener, pollDispatcher, deserializer, pollTimeoutDuration) + def newSocket(socketParameters: SocketOption*): ActorRef = { implicit val timeout = system.settings.ActorTimeout - val req = (zeromq ? Props(new ConcurrentSocketActor(params)).withDispatcher("akka.zeromq.socket-dispatcher")).mapTo[ActorRef] + val req = (zeromq ? newSocketProps(socketParameters: _*)).mapTo[ActorRef] Await.result(req, timeout.duration) } diff --git a/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala b/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala index 6c256dd58e..d29579d8cb 100644 --- a/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala +++ b/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala @@ -37,7 +37,7 @@ class ConcurrentSocketActorSpec "support pub-sub connections" in { checkZeroMQInstallation val (publisherProbe, subscriberProbe) = (TestProbe(), TestProbe()) - val context = zmq.newContext() + val context = Context() val publisher = newPublisher(context, publisherProbe.ref) val subscriber = newSubscriber(context, subscriberProbe.ref) val msgGenerator = newMessageGenerator(publisher) @@ -63,7 +63,7 @@ class ConcurrentSocketActorSpec "support zero-length message frames" in { checkZeroMQInstallation val publisherProbe = TestProbe() - val context = zmq.newContext() + val context = Context() val publisher = newPublisher(context, publisherProbe.ref) try { @@ -77,18 +77,18 @@ class ConcurrentSocketActorSpec } } def newPublisher(context: Context, listener: ActorRef) = { - val publisher = zmq.newSocket(SocketType.Pub, context = context, listener = Some(listener)) + val publisher = zmq.newSocket(SocketType.Pub, context, Listener(listener)) publisher ! Bind(endpoint) publisher } def newSubscriber(context: Context, listener: ActorRef) = { - val subscriber = zmq.newSocket(SocketType.Sub, context = context, listener = Some(listener)) + val subscriber = zmq.newSocket(SocketType.Sub, context, Listener(listener)) subscriber ! Connect(endpoint) subscriber ! Subscribe(Seq()) subscriber } def newMessageGenerator(actorRef: ActorRef) = { - system.actorOf(Props(new MessageGeneratorActor(actorRef)).withTimeout(Timeout(10 millis))) + system.actorOf(Props(new MessageGeneratorActor(actorRef))) } def checkZeroMQInstallation = try { @@ -114,7 +114,7 @@ class ConcurrentSocketActorSpec private var genMessages: Cancellable = null override def preStart() = { - genMessages = system.scheduler.schedule(100 millis, 10 millis, self, 'm) + genMessages = system.scheduler.schedule(100 millis, 10 millis, self, "genMessage") } override def postStop() = { From d2275ec4d2fc7314e623d3547b75d7b2e526e628 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Wed, 18 Jan 2012 18:01:33 -0700 Subject: [PATCH 036/152] IO ScalaDocs about half done --- akka-actor/src/main/scala/akka/actor/IO.scala | 265 +++++++++++++++++- 1 file changed, 253 insertions(+), 12 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/IO.scala b/akka-actor/src/main/scala/akka/actor/IO.scala index 5942f285de..1911770a63 100644 --- a/akka-actor/src/main/scala/akka/actor/IO.scala +++ b/akka-actor/src/main/scala/akka/actor/IO.scala @@ -23,8 +23,17 @@ import scala.annotation.tailrec import scala.collection.generic.CanBuildFrom import com.eaio.uuid.UUID +/** + * IO messages and iteratees. + */ object IO { + /** + * An immutable handle to a Java NIO Channel. Contains a reference to the + * [[akka.actor.ActorRef]] that will receive events related to the Channel, + * a reference to the [[akka.actor.IOManager]] that manages the Channel, and + * a [[com.eaio.uuid.UUID]] to uniquely identify the Channel. + */ sealed trait Handle { this: Product ⇒ def owner: ActorRef @@ -37,26 +46,69 @@ object IO { def asSocket: SocketHandle = sys error "Not a socket" def asServer: ServerHandle = sys error "Not a server" + /** + * Sends a request to the [[akka.actor.IOManager]] to close the Channel + * associated with this [[akka.actor.IO.Handle]]. + * + * This can also be performed by sending [[akka.actor.IO.Close]] to the + * [[akka.actor.IOManager]]. + */ def close(): Unit = ioManager ! Close(this) } + /** + * A [[akka.actor.IO.Handle]] to a ReadableByteChannel. + */ sealed trait ReadHandle extends Handle with Product { override def asReadable = this } + /** + * A [[akka.actor.IO.Handle]] to a WritableByteChannel. + */ sealed trait WriteHandle extends Handle with Product { override def asWritable = this + /** + * Sends a request to the [[akka.actor.IOManager]] to write to the + * Channel associated with this [[akka.actor.IO.Handle]]. + * + * This can also be performed by sending [[akka.actor.IO.Write]] to the + * [[akka.actor.IOManager]]. + */ def write(bytes: ByteString): Unit = ioManager ! Write(this, bytes) } + /** + * A [[akka.actor.IO.Handle]] to a SocketChannel. Instances are normally + * 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 } + /** + * 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 + /** + * Sends a request to the [[akka.actor.IOManager]] to accept an incoming + * connection to the ServerSocketChannel associated with this + * [[akka.actor.IO.Handle]]. + * + * This can also be performed by creating a new [[akka.actor.IO.SocketHandle]] + * and sending it within an [[akka.actor.IO.Accept]] to the [[akka.actor.IOManager]]. + * + * @param socketOwner the [[akka.actor.ActorRef]] that should receive events + * associated with the SocketChannel. The ActorRef for the + * current Actor will be used implicitly. + * @return a new SocketHandle that can be used to perform actions on the + * new connection's SocketChannel. + */ def accept()(implicit socketOwner: ActorRef): SocketHandle = { val socket = SocketHandle(socketOwner, ioManager) ioManager ! Accept(socket, this) @@ -64,18 +116,91 @@ object IO { } } + /** + * Messages used to communicate with an [[akka.actor.IOManager]]. + */ sealed trait IOMessage + + /** + * Message to an [[akka.actor.IOManager]] to create a ServerSocketChannel + * listening on the provided address. + * + * Normally sent using IOManager.listen() + */ case class Listen(server: ServerHandle, address: InetSocketAddress) extends IOMessage + + /** + * Message from an [[akka.actor.IOManager]] that a new connection has been + * made to the ServerSocketChannel and needs to be accepted. + */ case class NewClient(server: ServerHandle) extends IOMessage + + /** + * Message to an [[akka.actor.IOManager]] to accept a new connection. + * + * Normally sent using [[akka.actor.IO.ServerHandle]].accept() + */ case class Accept(socket: SocketHandle, server: ServerHandle) extends IOMessage + + /** + * Message to an [[akka.actor.IOManager]] to create a SocketChannel connected + * to the provided address. + * + * Normally sent using IOManager.connect() + */ case class Connect(socket: SocketHandle, address: InetSocketAddress) extends IOMessage + + /** + * Message from an [[akka.actor.IOManager]] that the SocketChannel has + * successfully connected. + * + * No action is required by the receiving [[akka.actor.Actor]], although + * the message still needs to be in it's receive method. + */ case class Connected(socket: SocketHandle) extends IOMessage + + /** + * Message to an [[akka.actor.IOManager]] to close the Channel. + * + * Normally sent using [[akka.actor.IO.Handle]].close() + */ case class Close(handle: Handle) extends IOMessage + + /** + * Message from an [[akka.actor.IOManager]] that the Channel has closed. Can + * optionally contain the Exception that caused the Channel to close, if + * applicable. + * + * No action is required by the receiving [[akka.actor.Actor]], although + * the message still needs to be in it's receive method. + */ case class Closed(handle: Handle, cause: Option[Exception]) extends IOMessage + + /** + * Message from an [[akka.actor.IOManager]] that contains bytes read from + * the SocketChannel. + */ case class Read(handle: ReadHandle, bytes: ByteString) extends IOMessage + + /** + * Message to an [[akka.actor.IOManager]] to write to the SocketChannel. + * + * Normally sent using [[akka.actor.IO.SocketHandle]].write() + */ case class Write(handle: WriteHandle, bytes: ByteString) extends IOMessage + /** + * Represents part of a stream of bytes that can be processed by an + * [[akka.actor.IO.Iteratee]]. + */ sealed trait Input { + /** + * Append another Input to this one. + * + * If 'that' is an [[akka.actor.IO.EOF]] then it will replace any + * remaining bytes in this Input. If 'this' is an [[akka.actor.IO.EOF]] + * then it will be replaced by 'that'. + */ def ++(that: Input): Input } @@ -83,6 +208,9 @@ object IO { val empty = 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) @@ -90,11 +218,24 @@ object IO { } } + /** + * Part of an [[akka.actor.IO.Input]] stream that represents the end of the + * stream. + * + * This will cause the [[akka.actor.IO.Iteratee]] that processes it + * to terminate early. If a cause is defined it can be 'caught' by + * Iteratee.recover() in order to handle it properly. + */ case class EOF(cause: Option[Exception]) extends Input { def ++(that: Input) = that } object Iteratee { + /** + * Wrap the provided value within a [[akka.actor.IO.Done]] + * [[akka.actor.IO.Iteratee]]. This is a helper for cases where the type should be + * inferred as an Iteratee and not as a Done. + */ def apply[A](value: A): Iteratee[A] = Done(value) def apply(): Iteratee[Unit] = unit val unit: Iteratee[Unit] = Done(()) @@ -102,37 +243,67 @@ object IO { /** * A basic Iteratee implementation of Oleg's Iteratee (http://okmij.org/ftp/Streams.html). - * No support for Enumerator or Input types other then ByteString at the moment. + * To keep this implementation simple it has no support for Enumerator or Input types + * other then ByteString. + * + * Other Iteratee implementations can be used in place of this one if any + * missing features are required. */ sealed abstract class Iteratee[+A] { /** - * Applies the given input to the Iteratee, returning the resulting Iteratee - * and the unused Input. + * Processes the given [[akka.actor.IO.Input]], returning the resulting + * Iteratee and the remaining Input. */ final def apply(input: Input): (Iteratee[A], Input) = this match { case Cont(f, None) ⇒ f(input) case iter ⇒ (iter, input) } + /** + * Passes an [[akka.actor.IO.EOF]] to this Iteratee and returns the + * result if available. + * + * If this Iteratee is in a failure state then the Exception will be thrown. + * + * If this Iteratee is not well behaved (does not return a result on EOF) + * then a "Divergent Iteratee" Exception will be thrown. + */ final def get: A = this(EOF(None))._1 match { case Done(value) ⇒ value case Cont(_, None) ⇒ sys.error("Divergent Iteratee") case Cont(_, Some(err)) ⇒ throw err } + /** + * Applies a function to the result of this Iteratee, resulting in a new + * Iteratee. Any unused [[akka.actor.IO.Input]] that is given to this + * Iteratee will be passed to that resulting Iteratee. This is the + * primary method of composing Iteratees together in order to process + * an Input stream. + */ final def flatMap[B](f: A ⇒ Iteratee[B]): Iteratee[B] = this match { case Done(value) ⇒ f(value) case Cont(k: Chain[_], err) ⇒ Cont(k :+ f, err) case Cont(k, err) ⇒ Cont(Chain(k, f), err) } + /** + * Applies a function to transform the result of this Iteratee. + */ final def map[B](f: A ⇒ B): Iteratee[B] = this match { case Done(value) ⇒ Done(f(value)) case Cont(k: Chain[_], err) ⇒ Cont(k :+ ((a: A) ⇒ Done(f(a))), err) case Cont(k, err) ⇒ Cont(Chain(k, (a: A) ⇒ Done(f(a))), err) } + /** + * Provides a handler for any matching errors that may have occured while + * running this Iteratee. + * + * Errors are usually raised within the Iteratee with [[akka.actor.IO]].throwErr + * or by processing an [[akka.actor.IO.EOF]] that contains an Exception. + */ def recover[B >: A](pf: PartialFunction[Exception, B]): Iteratee[B] = this match { case done @ Done(_) ⇒ done case Cont(_, Some(err)) if pf isDefinedAt err ⇒ Done(pf(err)) @@ -142,24 +313,54 @@ object IO { } /** - * An Iteratee representing a result and the remaining ByteString. Also used to - * wrap any constants or precalculated values that need to be composed with - * other Iteratees. + * An Iteratee representing a result, usually returned by the successful + * completion of an Iteratee. Also used to wrap any constants or + * precalculated values that need to be composed with other Iteratees. */ final case class Done[+A](result: A) extends Iteratee[A] /** - * An Iteratee that still requires more input to calculate it's result. + * An [[akka.actor.IO.Iteratee]] that still requires more input to calculate + * it's result. It may also contain an optional error, which can be handled + * with 'recover()'. + * + * It is possible to recover from an error and continue processing this + * Iteratee without losing the continuation, although that has not yet + * been tested. An example use case of this is resuming a failed download. */ final case class Cont[+A](f: Input ⇒ (Iteratee[A], Input), error: Option[Exception] = None) extends Iteratee[A] object IterateeRef { + + /** + * Creates an [[akka.actor.IO.IterateeRefSync]] containing an initial + * [[akka.actor.IO.Iteratee]]. + */ def sync[A](initial: Iteratee[A]): IterateeRefSync[A] = new IterateeRefSync(initial) + + /** + * Creates an empty [[akka.actor.IO.IterateeRefSync]]. + */ def sync(): IterateeRefSync[Unit] = new IterateeRefSync(Iteratee.unit) + /** + * Creates an [[akka.actor.IO.IterateeRefAsync]] containing an initial + * [[akka.actor.IO.Iteratee]]. + */ def async[A](initial: Iteratee[A])(implicit executor: ExecutionContext): IterateeRefAsync[A] = new IterateeRefAsync(initial) + + /** + * Creates an empty [[akka.actor.IO.IterateeRefAsync]]. + */ def async()(implicit executor: ExecutionContext): IterateeRefAsync[Unit] = new IterateeRefAsync(Iteratee.unit) + /** + * A mutable Map to contain multiple IterateeRefs. + * + * This Map differs from the mutable Map within Scala's standard library + * by automatically including any keys used to lookup an IterateeRef. The + * '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 @@ -167,20 +368,31 @@ object IO { def -=(key: K) = { underlying -= key; this } override def empty = new Map[K, V](refFactory) } + object Map { + /** + * Uses a factory to create the initial IterateeRef for each new key. + */ def apply[K, V](refFactory: ⇒ IterateeRef[V]): IterateeRef.Map[K, V] = new Map(refFactory) + + /** + * Creates an empty [[akka.actor.IO.IterateeRefSync]] for each new key. + */ def sync[K](): IterateeRef.Map[K, Unit] = new Map(IterateeRef.sync()) + + /** + * Creates an empty [[akka.actor.IO.IterateeRefAsync]] for each new key. + */ def async[K]()(implicit executor: ExecutionContext): IterateeRef.Map[K, Unit] = new Map(IterateeRef.async()) } + } /** - * A mutable reference to an Iteratee. Not thread safe. + * A mutable reference to an Iteratee designed for use within an Actor. * - * Designed for use within an Actor. - * - * Includes mutable implementations of flatMap, map, and apply which - * update the internal reference and return Unit. + * See [[akka.actor.IO.IterateeRefSync]] and [[akka.actor.IO.IterateeRefAsync]] + * for details. */ trait IterateeRef[A] { def flatMap(f: A ⇒ Iteratee[A]): Unit @@ -188,6 +400,17 @@ object IO { def apply(input: Input): Unit } + /** + * A mutable reference to an [[akka.actor.IO.Iteratee]]. Not thread safe. + * + * Designed for use within an [[akka.actor.Actor]]. + * + * Includes mutable implementations of flatMap, map, and apply which + * update the internal reference and return Unit. + * + * [[akka.actor.IO.Input]] remaining after processing the Iteratee will + * be stored and processed later when 'flatMap' is used next. + */ 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 { @@ -199,6 +422,20 @@ object IO { def value: (Iteratee[A], Input) = _value } + /** + * A mutable reference to an [[akka.actor.IO.Iteratee]]. Not thread safe. + * + * Designed for use within an [[akka.actor.Actor]], although all actions + * perfomed on the Iteratee are processed within a [[akka.dispatch.Future]] + * so it is not safe to refer to the Actor's state from within this Iteratee. + * Messages should instead be sent to the Actor in order to modify state. + * + * Includes mutable implementations of flatMap, map, and apply which + * update the internal reference and return Unit. + * + * [[akka.actor.IO.Input]] remaining after processing the Iteratee will + * be stored and processed later when 'flatMap' is used next. + */ 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 { @@ -210,6 +447,10 @@ object IO { def future: Future[(Iteratee[A], Input)] = _value } + /** + * An [[akka.actor.IO.Iteratee]] that contains an Exception. The Exception + * can be handled with Iteratee.recover(). + */ final def throwErr(err: Exception): Iteratee[Nothing] = Cont(input ⇒ (throwErr(err), input), Some(err)) /** From 03c547c1a1945ebc2c2241a3dc17227f33f31aa8 Mon Sep 17 00:00:00 2001 From: Ivan Porto Carrero Date: Thu, 19 Jan 2012 09:50:59 +0100 Subject: [PATCH 037/152] Also setup connection and subscriptions again after a restart --- .../akka/zeromq/ConcurrentSocketActor.scala | 87 ++++++++++--------- .../main/scala/akka/zeromq/Deserializer.scala | 4 - .../src/main/scala/akka/zeromq/Requests.scala | 33 ++++--- .../scala/akka/zeromq/ZeroMQExtension.scala | 2 - .../zeromq/ConcurrentSocketActorSpec.scala | 9 +- 5 files changed, 71 insertions(+), 64 deletions(-) diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala index 07a7b5f888..3f57138932 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala @@ -60,26 +60,45 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A } private def handleSocketOption: Receive = { - case Linger ⇒ sender ! socket.getLinger - case ReconnectIVL ⇒ sender ! socket.getReconnectIVL - case Backlog ⇒ sender ! socket.getBacklog - case ReconnectIVLMax ⇒ sender ! socket.getReconnectIVLMax - case MaxMsgSize ⇒ sender ! socket.getMaxMsgSize - case SndHWM ⇒ sender ! socket.getSndHWM - case RcvHWM ⇒ sender ! socket.getRcvHWM - case Swap ⇒ sender ! socket.getSwap - case Affinity ⇒ sender ! socket.getAffinity - case Identity ⇒ sender ! socket.getIdentity - case Rate ⇒ sender ! socket.getRate - case RecoveryInterval ⇒ sender ! socket.getRecoveryInterval - case MulticastLoop ⇒ sender ! socket.hasMulticastLoop - case MulticastHops ⇒ sender ! socket.getMulticastHops - case ReceiveTimeOut ⇒ sender ! socket.getReceiveTimeOut - case SendTimeOut ⇒ sender ! socket.getSendTimeOut - case SendBufferSize ⇒ sender ! socket.getSendBufferSize - case ReceiveBufferSize ⇒ sender ! socket.getReceiveBufferSize - case ReceiveMore ⇒ sender ! socket.hasReceiveMore - case FileDescriptor ⇒ sender ! socket.getFD + case Linger(value) ⇒ socket.setLinger(value) + case ReconnectIVL(value) ⇒ socket.setReconnectIVL(value) + case Backlog(value) ⇒ socket.setBacklog(value) + case ReconnectIVLMax(value) ⇒ socket.setReconnectIVLMax(value) + case MaxMsgSize(value) ⇒ socket.setMaxMsgSize(value) + case SndHWM(value) ⇒ socket.setSndHWM(value) + case RcvHWM(value) ⇒ socket.setRcvHWM(value) + case HWM(value) ⇒ socket.setHWM(value) + case Swap(value) ⇒ socket.setSwap(value) + case Affinity(value) ⇒ socket.setAffinity(value) + case Identity(value) ⇒ socket.setIdentity(value) + case Rate(value) ⇒ socket.setRate(value) + case RecoveryInterval(value) ⇒ socket.setRecoveryInterval(value) + case MulticastLoop(value) ⇒ socket.setMulticastLoop(value) + case MulticastHops(value) ⇒ socket.setMulticastHops(value) + case ReceiveTimeOut(value) ⇒ socket.setReceiveTimeOut(value) + case SendTimeOut(value) ⇒ socket.setSendTimeOut(value) + case SendBufferSize(value) ⇒ socket.setSendBufferSize(value) + case ReceiveBufferSize(value) ⇒ socket.setReceiveBufferSize(value) + case Linger ⇒ sender ! socket.getLinger + case ReconnectIVL ⇒ sender ! socket.getReconnectIVL + case Backlog ⇒ sender ! socket.getBacklog + case ReconnectIVLMax ⇒ sender ! socket.getReconnectIVLMax + case MaxMsgSize ⇒ sender ! socket.getMaxMsgSize + case SndHWM ⇒ sender ! socket.getSndHWM + case RcvHWM ⇒ sender ! socket.getRcvHWM + case Swap ⇒ sender ! socket.getSwap + case Affinity ⇒ sender ! socket.getAffinity + case Identity ⇒ sender ! socket.getIdentity + case Rate ⇒ sender ! socket.getRate + case RecoveryInterval ⇒ sender ! socket.getRecoveryInterval + case MulticastLoop ⇒ sender ! socket.hasMulticastLoop + case MulticastHops ⇒ sender ! socket.getMulticastHops + case ReceiveTimeOut ⇒ sender ! socket.getReceiveTimeOut + case SendTimeOut ⇒ sender ! socket.getSendTimeOut + case SendBufferSize ⇒ sender ! socket.getSendBufferSize + case ReceiveBufferSize ⇒ sender ! socket.getReceiveBufferSize + case ReceiveMore ⇒ sender ! socket.hasReceiveMore + case FileDescriptor ⇒ sender ! socket.getFD } private def internalMessage: Receive = { @@ -106,6 +125,12 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A override def preStart { setupSocket() poller.register(socket, Poller.POLLIN) + setupConnection() + } + + private def setupConnection() { + params filter (_.isInstanceOf[SocketConnectOption]) foreach { self ! _ } + params filter (_.isInstanceOf[PubSubOption]) foreach { self ! _ } } private def socketFromParams() = { @@ -121,26 +146,8 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A private def setupSocket() = { params foreach { - case Linger(value) ⇒ socket.setLinger(value) - case ReconnectIVL(value) ⇒ socket.setReconnectIVL(value) - case Backlog(value) ⇒ socket.setBacklog(value) - case ReconnectIVLMax(value) ⇒ socket.setReconnectIVLMax(value) - case MaxMsgSize(value) ⇒ socket.setMaxMsgSize(value) - case SndHWM(value) ⇒ socket.setSndHWM(value) - case RcvHWM(value) ⇒ socket.setRcvHWM(value) - case HWM(value) ⇒ socket.setHWM(value) - case Swap(value) ⇒ socket.setSwap(value) - case Affinity(value) ⇒ socket.setAffinity(value) - case Identity(value) ⇒ socket.setIdentity(value) - case Rate(value) ⇒ socket.setRate(value) - case RecoveryInterval(value) ⇒ socket.setRecoveryInterval(value) - case MulticastLoop(value) ⇒ socket.setMulticastLoop(value) - case MulticastHops(value) ⇒ socket.setMulticastHops(value) - case ReceiveTimeOut(value) ⇒ socket.setReceiveTimeOut(value) - case SendTimeOut(value) ⇒ socket.setSendTimeOut(value) - case SendBufferSize(value) ⇒ socket.setSendBufferSize(value) - case ReceiveBufferSize(value) ⇒ socket.setReceiveBufferSize(value) - case _ ⇒ + case _: SocketConnectOption | _: PubSubOption | _: SocketMeta ⇒ // ignore, handled differently + case m ⇒ self ! m } } diff --git a/akka-zeromq/src/main/scala/akka/zeromq/Deserializer.scala b/akka-zeromq/src/main/scala/akka/zeromq/Deserializer.scala index daf75f7c4b..b9913ea560 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/Deserializer.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/Deserializer.scala @@ -6,10 +6,6 @@ package akka.zeromq case class Frame(payload: Seq[Byte]) object Frame { def apply(s: String): Frame = Frame(s.getBytes) } -trait Deserializer extends SocketOption { - def apply(frames: Seq[Frame]): Any -} - class ZMQMessageDeserializer extends Deserializer { def apply(frames: Seq[Frame]) = ZMQMessage(frames) } diff --git a/akka-zeromq/src/main/scala/akka/zeromq/Requests.scala b/akka-zeromq/src/main/scala/akka/zeromq/Requests.scala index 0b1a2dad90..371d2a3dde 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/Requests.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/Requests.scala @@ -10,15 +10,22 @@ import akka.util.duration._ import akka.util.Duration sealed trait Request -trait SocketOption extends Request +sealed trait SocketOption extends Request +sealed trait SocketMeta extends SocketOption +sealed trait SocketConnectOption extends SocketOption { + def endpoint: String +} +sealed trait PubSubOption extends SocketOption { + def payload: Seq[Byte] +} sealed trait SocketOptionQuery extends Request -case class Connect(endpoint: String) extends Request +case class Connect(endpoint: String) extends SocketConnectOption object Context { def apply(numIoThreads: Int = 1) = new Context(numIoThreads) } -class Context(numIoThreads: Int) extends SocketOption { +class Context(numIoThreads: Int) extends SocketMeta { private val context = JZMQ.context(numIoThreads) def socket(socketType: SocketType.ZMQSocketType) = { context.socket(socketType.id) @@ -31,27 +38,31 @@ class Context(numIoThreads: Int) extends SocketOption { } } +trait Deserializer extends SocketOption { + def apply(frames: Seq[Frame]): Any +} + object SocketType { - abstract class ZMQSocketType(val id: Int) extends SocketOption + abstract class ZMQSocketType(val id: Int) extends SocketMeta object Pub extends ZMQSocketType(JZMQ.PUB) object Sub extends ZMQSocketType(JZMQ.SUB) object Dealer extends ZMQSocketType(JZMQ.DEALER) object Router extends ZMQSocketType(JZMQ.ROUTER) } -case class Listener(listener: ActorRef) extends SocketOption -case class PollDispatcher(name: String) extends SocketOption -case class PollTimeoutDuration(duration: Duration = 100 millis) extends SocketOption +case class Listener(listener: ActorRef) extends SocketMeta +case class PollDispatcher(name: String) extends SocketMeta +case class PollTimeoutDuration(duration: Duration = 100 millis) extends SocketMeta -case class Bind(endpoint: String) extends Request +case class Bind(endpoint: String) extends SocketConnectOption private[zeromq] case object Close extends Request -case class Subscribe(payload: Seq[Byte]) extends Request +case class Subscribe(payload: Seq[Byte]) extends PubSubOption object Subscribe { - def apply(topic: String): Subscribe = Subscribe(topic.getBytes) + def apply(topic: String): Subscribe = new Subscribe(topic.getBytes) } -case class Unsubscribe(payload: Seq[Byte]) extends Request +case class Unsubscribe(payload: Seq[Byte]) extends PubSubOption object Unsubscribe { def apply(topic: String): Unsubscribe = Unsubscribe(topic.getBytes) } diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala index 685e3106bd..54d32c3264 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala @@ -3,12 +3,10 @@ */ package akka.zeromq -import akka.util.Duration import akka.util.duration._ import org.zeromq.{ ZMQ ⇒ JZMQ } import akka.actor._ import akka.dispatch.{ Dispatcher, Await } -import collection.mutable.ListBuffer case class ZeroMQVersion(major: Int, minor: Int, patch: Int) { override def toString = "%d.%d.%d".format(major, minor, patch) diff --git a/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala b/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala index d29579d8cb..844df2c139 100644 --- a/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala +++ b/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala @@ -77,15 +77,10 @@ class ConcurrentSocketActorSpec } } def newPublisher(context: Context, listener: ActorRef) = { - val publisher = zmq.newSocket(SocketType.Pub, context, Listener(listener)) - publisher ! Bind(endpoint) - publisher + zmq.newSocket(SocketType.Pub, context, Listener(listener), Bind(endpoint)) } def newSubscriber(context: Context, listener: ActorRef) = { - val subscriber = zmq.newSocket(SocketType.Sub, context, Listener(listener)) - subscriber ! Connect(endpoint) - subscriber ! Subscribe(Seq()) - subscriber + zmq.newSocket(SocketType.Sub, context, Listener(listener), Connect(endpoint), Subscribe(Seq.empty)) } def newMessageGenerator(actorRef: ActorRef) = { system.actorOf(Props(new MessageGeneratorActor(actorRef))) From 6f94c57ba673b990602c21fcd2f797245f8b85df Mon Sep 17 00:00:00 2001 From: Ivan Porto Carrero Date: Thu, 19 Jan 2012 12:04:35 +0100 Subject: [PATCH 038/152] move private objects for the ConcurrentSocketActor to a companion object --- .../scala/akka/zeromq/ConcurrentSocketActor.scala | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala index 3f57138932..50d1ee4c8a 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala @@ -15,8 +15,16 @@ private[zeromq] case object NoResults extends PollLifeCycle private[zeromq] case object Results extends PollLifeCycle private[zeromq] case object Closing extends PollLifeCycle +private[zeromq] object ConcurrentSocketActor { + private case object Poll + private case object ReceiveFrames + private case object ClearPoll + private case class PollError(ex: Throwable) + +} private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends Actor { + import ConcurrentSocketActor._ private val noBytes = Array[Byte]() private val zmqContext = { params find (_.isInstanceOf[Context]) map (_.asInstanceOf[Context]) getOrElse new Context(1) @@ -26,10 +34,6 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A private lazy val poller: Poller = zmqContext.poller private val log = Logging(context.system, this) - private case object Poll - private case object ReceiveFrames - private case object ClearPoll - private case class PollError(ex: Throwable) private def handleConnectionMessages: Receive = { case Send(frames) ⇒ { From 656a98fcafbf679feff0702bffd545f54420b792 Mon Sep 17 00:00:00 2001 From: Ivan Porto Carrero Date: Thu, 19 Jan 2012 12:38:36 +0100 Subject: [PATCH 039/152] uses collect first where appropriate, removes redundant config from the spec and uses a deathwatch for the listener --- .../akka/zeromq/ConcurrentSocketActor.scala | 37 +++++++++---------- .../zeromq/ConcurrentSocketActorSpec.scala | 5 --- 2 files changed, 18 insertions(+), 24 deletions(-) diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala index 50d1ee4c8a..2a0d376393 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala @@ -9,6 +9,7 @@ import akka.actor._ import akka.dispatch.{ Promise, Future } import akka.event.Logging import akka.util.duration._ +import annotation.tailrec private[zeromq] sealed trait PollLifeCycle private[zeromq] case object NoResults extends PollLifeCycle @@ -27,13 +28,13 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A import ConcurrentSocketActor._ private val noBytes = Array[Byte]() private val zmqContext = { - params find (_.isInstanceOf[Context]) map (_.asInstanceOf[Context]) getOrElse new Context(1) + params collectFirst { case c: Context ⇒ c } getOrElse Context() } - private lazy val deserializer = deserializerFromParams - private lazy val socket: Socket = socketFromParams - private lazy val poller: Poller = zmqContext.poller - private val log = Logging(context.system, this) + private val deserializer = deserializerFromParams + private val socket: Socket = socketFromParams + private val poller: Poller = zmqContext.poller + private val log = Logging(context.system, this) private def handleConnectionMessages: Receive = { case Send(frames) ⇒ { @@ -61,6 +62,7 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A socket.unsubscribe(topic.toArray) pollAndReceiveFrames() } + case Terminated(_) ⇒ context stop self } private def handleSocketOption: Receive = { @@ -127,6 +129,7 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A override def receive: Receive = handleConnectionMessages orElse handleSocketOption orElse internalMessage override def preStart { + watchListener() setupSocket() poller.register(socket, Poller.POLLIN) setupConnection() @@ -139,13 +142,11 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A private def socketFromParams() = { require(ZeroMQExtension.check[SocketType.ZMQSocketType](params), "A socket type is required") - (params - find (_.isInstanceOf[SocketType.ZMQSocketType]) - map (t ⇒ zmqContext.socket(t.asInstanceOf[SocketType.ZMQSocketType])) get) + (params collectFirst { case t: SocketType.ZMQSocketType ⇒ zmqContext.socket(t) } get) } private def deserializerFromParams = { - params find (_.isInstanceOf[Deserializer]) map (_.asInstanceOf[Deserializer]) getOrElse new ZMQMessageDeserializer + params collectFirst { case d: Deserializer ⇒ d } getOrElse new ZMQMessageDeserializer } private def setupSocket() = { @@ -181,14 +182,12 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A } private lazy val eventLoopDispatcher = { - val fromConfig = params.find(_.isInstanceOf[PollDispatcher]) map { - option ⇒ context.system.dispatchers.lookup(option.asInstanceOf[PollDispatcher].name) - } + val fromConfig = params collectFirst { case PollDispatcher(name) ⇒ context.system.dispatchers.lookup(name) } fromConfig getOrElse context.system.dispatcher } private lazy val pollTimeout = { - val fromConfig = params find (_.isInstanceOf[PollTimeoutDuration]) map (_.asInstanceOf[PollTimeoutDuration].duration) + val fromConfig = params collectFirst { case PollTimeoutDuration(duration) ⇒ duration } fromConfig getOrElse 100.millis } @@ -225,12 +224,12 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A } } + private def listenerOpt = params collectFirst { case Listener(l) ⇒ l } + private def watchListener() { + listenerOpt foreach context.watch + } + private def notifyListener(message: Any) { - params find (_.isInstanceOf[Listener]) map (_.asInstanceOf[Listener].listener) foreach { listener ⇒ - if (listener.isTerminated) - context stop self - else - listener ! message - } + listenerOpt foreach { _ ! message } } } diff --git a/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala b/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala index 844df2c139..4b3c5b46b6 100644 --- a/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala +++ b/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala @@ -15,11 +15,6 @@ object ConcurrentSocketActorSpec { val config = """ akka { extensions = ["akka.zeromq.ZeroMQExtension$"] - zeromq { - socket-dispatcher { - type = "PinnedDispatcher" - } - } } """ } From 69ace98b78e7d60e27324e5240d8fc8194f5492d Mon Sep 17 00:00:00 2001 From: Ivan Porto Carrero Date: Thu, 19 Jan 2012 12:41:04 +0100 Subject: [PATCH 040/152] use tailrecursive loop for reading the incoming zmq message --- .../akka/zeromq/ConcurrentSocketActor.scala | 20 +++++-------------- 1 file changed, 5 insertions(+), 15 deletions(-) diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala index 2a0d376393..7d00c7daea 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala @@ -205,23 +205,13 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A } private def receiveFrames(): Seq[Frame] = { - - @inline def receiveBytes(): Array[Byte] = socket.recv(0) match { - case null ⇒ noBytes - case bytes: Array[_] if bytes.length > 0 ⇒ bytes - case _ ⇒ noBytes + @tailrec def receiveBytes(next: Array[Byte], currentFrames: Vector[Frame] = Vector.empty): Seq[Frame] = { + val nwBytes = if (next != null && next.nonEmpty) next else noBytes + val frames = currentFrames :+ Frame(nwBytes) + if (socket.hasReceiveMore) receiveBytes(socket.recv(0), frames) else frames } - receiveBytes() match { - case `noBytes` ⇒ Vector.empty - case someBytes ⇒ - var frames = Vector(Frame(someBytes)) - while (socket.hasReceiveMore) receiveBytes() match { - case `noBytes` ⇒ - case someBytes ⇒ frames :+= Frame(someBytes) - } - frames - } + receiveBytes(socket.recv(0)) } private def listenerOpt = params collectFirst { case Listener(l) ⇒ l } From 6bdffd94a9b4b0b75ceb8beb77dbd4aaf73b712b Mon Sep 17 00:00:00 2001 From: Ivan Porto Carrero Date: Thu, 19 Jan 2012 12:50:51 +0100 Subject: [PATCH 041/152] make listener a val, remove other lazy vals --- .../main/scala/akka/zeromq/ConcurrentSocketActor.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala index 7d00c7daea..b016370234 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala @@ -158,8 +158,8 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A override def postStop { try { - poller.unregister(socket) currentPoll foreach { _ complete Right(Closing) } + poller.unregister(socket) if (socket != null) socket.close } finally { notifyListener(Closed) @@ -181,12 +181,12 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A if (currentPoll.isEmpty) currentPoll = newEventLoop } - private lazy val eventLoopDispatcher = { + private val eventLoopDispatcher = { val fromConfig = params collectFirst { case PollDispatcher(name) ⇒ context.system.dispatchers.lookup(name) } fromConfig getOrElse context.system.dispatcher } - private lazy val pollTimeout = { + private val pollTimeout = { val fromConfig = params collectFirst { case PollTimeoutDuration(duration) ⇒ duration } fromConfig getOrElse 100.millis } @@ -214,7 +214,7 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A receiveBytes(socket.recv(0)) } - private def listenerOpt = params collectFirst { case Listener(l) ⇒ l } + private val listenerOpt = params collectFirst { case Listener(l) ⇒ l } private def watchListener() { listenerOpt foreach context.watch } From 6600290cbe638ffb2c165445a268aae6ee924826 Mon Sep 17 00:00:00 2001 From: Ivan Porto Carrero Date: Thu, 19 Jan 2012 13:06:22 +0100 Subject: [PATCH 042/152] make poll timeout configurable --- akka-zeromq/src/main/resources/reference.conf | 3 +++ .../src/main/scala/akka/zeromq/ConcurrentSocketActor.scala | 7 ++++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/akka-zeromq/src/main/resources/reference.conf b/akka-zeromq/src/main/resources/reference.conf index ce562b9449..54922b8386 100644 --- a/akka-zeromq/src/main/resources/reference.conf +++ b/akka-zeromq/src/main/resources/reference.conf @@ -9,6 +9,9 @@ akka { zeromq { + # The default timeout for a poll on the actual zeromq socket. + poll-timeout = 100ms + socket-dispatcher { # A zeromq socket needs to be pinned to the thread that created it. # Changing this value results in weird errors and race conditions within zeromq diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala index b016370234..8585883868 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala @@ -10,6 +10,8 @@ import akka.dispatch.{ Promise, Future } import akka.event.Logging import akka.util.duration._ import annotation.tailrec +import akka.util.Duration +import java.util.concurrent.TimeUnit private[zeromq] sealed trait PollLifeCycle private[zeromq] case object NoResults extends PollLifeCycle @@ -186,9 +188,12 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A fromConfig getOrElse context.system.dispatcher } + private val defaultPollTimeout = + Duration(context.system.settings.config.getMilliseconds("akka.zeromq.poll-timeout"), TimeUnit.MILLISECONDS) + private val pollTimeout = { val fromConfig = params collectFirst { case PollTimeoutDuration(duration) ⇒ duration } - fromConfig getOrElse 100.millis + fromConfig getOrElse defaultPollTimeout } private def newEventLoop: Option[Promise[PollLifeCycle]] = { From 44c7f49f92c39540dd0cb0f17bb6e515dd57d355 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 19 Jan 2012 13:50:02 +0100 Subject: [PATCH 043/152] Removed FState, switched to isCompleted checks instead of value.isDefined, added zip etc --- .../test/scala/akka/dispatch/FutureSpec.scala | 30 ++++++- .../src/main/scala/akka/dispatch/Future.scala | 88 ++++++++++--------- 2 files changed, 74 insertions(+), 44 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala index f8aa76a7ca..943a24025b 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala @@ -11,11 +11,11 @@ import akka.testkit.{ EventFilter, filterEvents, filterException } import akka.util.duration._ import akka.testkit.AkkaSpec import org.scalatest.junit.JUnitSuite -import java.lang.ArithmeticException import akka.testkit.DefaultTimeout import akka.testkit.TestLatch import java.util.concurrent.{ TimeoutException, TimeUnit, CountDownLatch } import scala.runtime.NonLocalReturnControl +import java.lang.{ IllegalStateException, ArithmeticException } object FutureSpec { class TestActor extends Actor { @@ -327,6 +327,24 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa Await.result(Future.fold(futures)(0)(_ + _), timeout millis) must be(45) } + "zip" in { + val timeout = 10000 millis + val f = new IllegalStateException("test") + intercept[IllegalStateException] { + Await.result(Promise.failed[String](f) zip Promise.successful("foo"), timeout) + } must be(f) + + intercept[IllegalStateException] { + Await.result(Promise.successful("foo") zip Promise.failed[String](f), timeout) + } must be(f) + + intercept[IllegalStateException] { + Await.result(Promise.failed[String](f) zip Promise.failed[String](f), timeout) + } must be(f) + + Await.result(Promise.successful("foo") zip Promise.successful("foo"), timeout) must be(("foo", "foo")) + } + "fold by composing" in { val actors = (1 to 10).toList map { _ ⇒ system.actorOf(Props(new Actor { @@ -859,6 +877,12 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa Await.result(p, timeout.duration) must be(result) } } + "zip properly" in { + f { (future, result) ⇒ + Await.result(future zip Promise.successful("foo"), timeout.duration) must be((result, "foo")) + (evaluating { Await.result(future zip Promise.failed(new RuntimeException("ohnoes")), timeout.duration) } must produce[RuntimeException]).getMessage must be("ohnoes") + } + } "not recover from exception" in { f((future, result) ⇒ Await.result(future.recover({ case _ ⇒ "pigdog" }), timeout.duration) must be(result)) } "perform action on result" in { f { (future, result) ⇒ @@ -892,6 +916,10 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa "retain exception with map" in { f((future, message) ⇒ (evaluating { Await.result(future map (_.toString.length), timeout.duration) } must produce[E]).getMessage must be(message)) } "retain exception with flatMap" in { f((future, message) ⇒ (evaluating { Await.result(future flatMap (_ ⇒ Promise.successful[Any]("foo")), timeout.duration) } must produce[E]).getMessage must be(message)) } "not perform action with foreach" is pending + + "zip properly" in { + f { (future, message) ⇒ (evaluating { Await.result(future zip Promise.successful("foo"), timeout.duration) } must produce[E]).getMessage must be(message) } + } "recover from exception" in { f((future, message) ⇒ Await.result(future.recover({ case e if e.getMessage == message ⇒ "pigdog" }), timeout.duration) must be("pigdog")) } "not perform action on result" is pending "project a failure" in { f((future, message) ⇒ Await.result(future.failed, timeout.duration).getMessage must be(message)) } diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 452bf02927..17d7e095b7 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -346,6 +346,21 @@ sealed trait Future[+T] extends japi.Future[T] with Await.Awaitable[T] { case _ ⇒ source } + /** + * @returns a new Future that will contain a tuple containing the successful result of this and that Future. + * If this or that fail, they will race to complete the returned Future with their failure. + * The returned Future will not be completed if neither this nor that are completed. + */ + def zip[U](that: Future[U]): Future[(T, U)] = { + val p = Promise[(T, U)]() + onComplete { + case Left(t) ⇒ p failure t + case Right(r) ⇒ that onSuccess { case r2 ⇒ p success ((r, r2)) } + } + that onFailure { case f ⇒ p failure f } + p + } + /** * For use only within a Future.flow block or another compatible Delimited Continuations reset block. * @@ -357,7 +372,7 @@ sealed trait Future[+T] extends japi.Future[T] with Await.Awaitable[T] { /** * Tests whether this Future has been completed. */ - final def isCompleted: Boolean = value.isDefined + def isCompleted: Boolean /** * The contained value of this Future. Before this Future is completed @@ -676,23 +691,7 @@ trait Promise[T] extends Future[T] { //Companion object to FState, just to provide a cheap, immutable default entry private[dispatch] object DefaultPromise { - def EmptyPending[T](): FState[T] = emptyPendingValue.asInstanceOf[FState[T]] - - /** - * Represents the internal state of the DefaultCompletableFuture - */ - - sealed trait FState[+T] { def value: Option[Either[Throwable, T]] } - case class Pending[T](listeners: List[Either[Throwable, T] ⇒ Unit] = Nil) extends FState[T] { - def value: Option[Either[Throwable, T]] = None - } - case class Success[T](value: Option[Either[Throwable, T]] = None) extends FState[T] { - def result: T = value.get.right.get - } - case class Failure[T](value: Option[Either[Throwable, T]] = None) extends FState[T] { - def exception: Throwable = value.get.left.get - } - private val emptyPendingValue = Pending[Nothing](Nil) + def EmptyPending[T](): List[T] = Nil } /** @@ -701,28 +700,25 @@ private[dispatch] object DefaultPromise { class DefaultPromise[T](implicit val executor: ExecutionContext) extends AbstractPromise with Promise[T] { self ⇒ - import DefaultPromise.{ FState, Success, Failure, Pending } - protected final def tryAwait(atMost: Duration): Boolean = { Future.blocking @tailrec def awaitUnsafe(waitTimeNanos: Long): Boolean = { - if (value.isEmpty && waitTimeNanos > 0) { + if (!isCompleted && waitTimeNanos > 0) { val ms = NANOSECONDS.toMillis(waitTimeNanos) val ns = (waitTimeNanos % 1000000l).toInt //As per object.wait spec val start = System.nanoTime() - try { synchronized { if (value.isEmpty) wait(ms, ns) } } catch { case e: InterruptedException ⇒ } + try { synchronized { if (!isCompleted) wait(ms, ns) } } catch { case e: InterruptedException ⇒ } awaitUnsafe(waitTimeNanos - (System.nanoTime() - start)) - } else - value.isDefined + } else isCompleted } awaitUnsafe(if (atMost.isFinite) atMost.toNanos else Long.MaxValue) } def ready(atMost: Duration)(implicit permit: CanAwait): this.type = - if (value.isDefined || tryAwait(atMost)) this + if (isCompleted || tryAwait(atMost)) this else throw new TimeoutException("Futures timed out after [" + atMost.toMillis + "] milliseconds") def result(atMost: Duration)(implicit permit: CanAwait): T = @@ -731,16 +727,24 @@ class DefaultPromise[T](implicit val executor: ExecutionContext) extends Abstrac case Right(r) ⇒ r } - def value: Option[Either[Throwable, T]] = getState.value + def value: Option[Either[Throwable, T]] = getState match { + case _: List[_] ⇒ None + case c: Either[_, _] ⇒ Some(c.asInstanceOf[Either[Throwable, T]]) + } + + def isCompleted(): Boolean = getState match { + case _: Either[_, _] ⇒ true + case _ ⇒ false + } @inline - private[this] final def updater = AbstractPromise.updater.asInstanceOf[AtomicReferenceFieldUpdater[AbstractPromise, FState[T]]] + private[this] final def updater = AbstractPromise.updater.asInstanceOf[AtomicReferenceFieldUpdater[AbstractPromise, AnyRef]] @inline - protected final def updateState(oldState: FState[T], newState: FState[T]): Boolean = updater.compareAndSet(this, oldState, newState) + protected final def updateState(oldState: AnyRef, newState: AnyRef): Boolean = updater.compareAndSet(this, oldState, newState) @inline - protected final def getState: FState[T] = updater.get(this) + protected final def getState: AnyRef = updater.get(this) def tryComplete(value: Either[Throwable, T]): Boolean = { val callbacks: List[Either[Throwable, T] ⇒ Unit] = { @@ -748,9 +752,9 @@ class DefaultPromise[T](implicit val executor: ExecutionContext) extends Abstrac @tailrec def tryComplete(v: Either[Throwable, T]): List[Either[Throwable, T] ⇒ Unit] = { getState match { - case cur @ Pending(listeners) ⇒ - if (updateState(cur, if (v.isLeft) Failure(Some(v)) else Success(Some(v)))) listeners - else tryComplete(v) + case raw: List[_] ⇒ + val cur = raw.asInstanceOf[List[Either[Throwable, T] ⇒ Unit]] + if (updateState(cur, v)) cur else tryComplete(v) case _ ⇒ null } } @@ -769,22 +773,20 @@ class DefaultPromise[T](implicit val executor: ExecutionContext) extends Abstrac def onComplete(func: Either[Throwable, T] ⇒ Unit): this.type = { @tailrec //Returns whether the future has already been completed or not - def tryAddCallback(): Boolean = { + def tryAddCallback(): Either[Throwable, T] = { val cur = getState cur match { - case _: Success[_] | _: Failure[_] ⇒ true - case p: Pending[_] ⇒ - val pt = p.asInstanceOf[Pending[T]] - if (updateState(pt, pt.copy(listeners = func :: pt.listeners))) false else tryAddCallback() + case r: Either[_, _] ⇒ r.asInstanceOf[Either[Throwable, T]] + case listeners: List[_] ⇒ if (updateState(listeners, func :: listeners)) null else tryAddCallback() } } - if (tryAddCallback()) { - val result = value.get - Future.dispatchTask(() ⇒ notifyCompleted(func, result)) + tryAddCallback() match { + case null ⇒ this + case completed ⇒ + Future.dispatchTask(() ⇒ notifyCompleted(func, completed)) + this } - - this } private final def notifyCompleted(func: Either[Throwable, T] ⇒ Unit, result: Either[Throwable, T]) { @@ -805,7 +807,7 @@ final class KeptPromise[T](suppliedValue: Either[Throwable, T])(implicit val exe Future dispatchTask (() ⇒ func(completedAs)) this } - + def isCompleted(): Boolean = true def ready(atMost: Duration)(implicit permit: CanAwait): this.type = this def result(atMost: Duration)(implicit permit: CanAwait): T = value.get match { case Left(e) ⇒ throw e From 19347dadbc31e905d2dd1ed7338fa0c4c0f7ba39 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 19 Jan 2012 15:49:44 +0100 Subject: [PATCH 044/152] Renaming TestActorRef.apply to receive, in order to get forwarders generated & improve Java API --- akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala | 4 ++-- akka-docs/scala/testing.rst | 4 ++-- akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala | 2 +- .../src/test/scala/akka/testkit/TestActorRefSpec.scala | 4 ++-- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala b/akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala index 0678860ba9..8eaba57c8c 100644 --- a/akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala @@ -132,7 +132,7 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender { import akka.testkit.TestActorRef system.eventStream.subscribe(testActor, classOf[UnhandledMessage]) val ref = TestActorRef[MyActor] - ref(Unknown) + ref.receive(Unknown) expectMsg(1 second, UnhandledMessage(Unknown, system.deadLetters, ref)) //#test-unhandled } @@ -146,7 +146,7 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender { case boom ⇒ throw new IllegalArgumentException("boom") } }) - intercept[IllegalArgumentException] { actorRef("hello") } + intercept[IllegalArgumentException] { actorRef.receive("hello") } //#test-expecting-exceptions } diff --git a/akka-docs/scala/testing.rst b/akka-docs/scala/testing.rst index 47157a2385..0f80342bc4 100644 --- a/akka-docs/scala/testing.rst +++ b/akka-docs/scala/testing.rst @@ -75,7 +75,7 @@ Expecting Exceptions -------------------- Testing that an expected exception is thrown while processing a message sent to -the actor under test can be done by using a :class:`TestActorRef` :meth:`apply` based +the actor under test can be done by using a :class:`TestActorRef` :meth:`receive` based invocation: .. includecode:: code/akka/docs/testkit/TestkitDocSpec.scala#test-expecting-exceptions @@ -144,7 +144,7 @@ The Way In-Between If you want to test the actor behavior, including hotswapping, but without involving a dispatcher and without having the :class:`TestActorRef` swallow any thrown exceptions, then there is another mode available for you: just use -the :meth:`apply` method :class:`TestActorRef`, which will be forwarded to the +the :meth:`receive` method :class:`TestActorRef`, which will be forwarded to the underlying actor: .. includecode:: code/akka/docs/testkit/TestkitDocSpec.scala#test-unhandled diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala index eaeecf7487..85057790d2 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala @@ -58,7 +58,7 @@ class TestActorRef[T <: Actor]( * thrown will be available to you, while still being able to use * become/unbecome. */ - def apply(o: Any) { underlyingActor.apply(o) } + def receive(o: Any) { underlyingActor.apply(o) } /** * Retrieve reference to the underlying actor, where the static type matches the factory used inside the diff --git a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala index d5c9b1a151..95082304db 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala @@ -224,9 +224,9 @@ class TestActorRefSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTime a.underlying.dispatcher.getClass must be(classOf[CallingThreadDispatcher]) } - "proxy apply for the underlying actor" in { + "proxy receive for the underlying actor" in { val ref = TestActorRef[WorkerActor] - ref("work") + ref.receive("work") ref.isTerminated must be(true) } From 69196a998f6d699c54183fd11187a45f180a237c Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 19 Jan 2012 16:35:36 +0100 Subject: [PATCH 045/152] Switching to semi-dynamic discovery of sun.misc.Unsafe entity --- akka-actor/src/main/scala/akka/util/Unsafe.java | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/akka-actor/src/main/scala/akka/util/Unsafe.java b/akka-actor/src/main/scala/akka/util/Unsafe.java index 4449f045be..b1d7f61290 100644 --- a/akka-actor/src/main/scala/akka/util/Unsafe.java +++ b/akka-actor/src/main/scala/akka/util/Unsafe.java @@ -11,9 +11,16 @@ public final class Unsafe { public final static sun.misc.Unsafe instance; static { try { - Field field = sun.misc.Unsafe.class.getDeclaredField("theUnsafe"); - field.setAccessible(true); - instance = (sun.misc.Unsafe) field.get(null); + sun.misc.Unsafe found = null; + for(Field field : sun.misc.Unsafe.class.getDeclaredFields()) { + if (field.getType() == sun.misc.Unsafe.class) { + field.setAccessible(true); + found = (sun.misc.Unsafe) field.get(null); + break; + } + } + if (found == null) throw new IllegalStateException("Can't find instance of sun.misc.Unsafe"); + else instance = found; } catch(Throwable t) { throw new ExceptionInInitializerError(t); } From cc1a22c902b0bd269f03f322711cdb38dcb9e6a6 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 19 Jan 2012 17:41:53 +0100 Subject: [PATCH 046/152] Fixing signature of Futures.reduce --- akka-actor/src/main/scala/akka/dispatch/Future.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 452bf02927..35eb49e24d 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -96,14 +96,14 @@ object Futures { * or the result of the fold. */ def fold[T <: AnyRef, R <: AnyRef](zero: R, futures: JIterable[Future[T]], fun: akka.japi.Function2[R, T, R], executor: ExecutionContext): Future[R] = - Future.fold(scala.collection.JavaConversions.iterableAsScalaIterable(futures))(zero)(fun.apply _)(executor) + Future.fold(scala.collection.JavaConversions.iterableAsScalaIterable(futures))(zero)(fun.apply)(executor) /** * Java API. * Initiates a fold over the supplied futures where the fold-zero is the result value of the Future that's completed first */ - def reduce[T <: AnyRef, R >: T](futures: JIterable[Future[T]], fun: akka.japi.Function2[R, R, T], executor: ExecutionContext): Future[R] = - Future.reduce(scala.collection.JavaConversions.iterableAsScalaIterable(futures))(fun.apply _)(executor) + def reduce[T <: AnyRef, R >: T](futures: JIterable[Future[T]], fun: akka.japi.Function2[R, T, R], executor: ExecutionContext): Future[R] = + Future.reduce[T, R](scala.collection.JavaConversions.iterableAsScalaIterable(futures))(fun.apply)(executor) /** * Java API. From 290f508195ad7934a32a8f8586b96d315e4dbe80 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 19 Jan 2012 17:47:58 +0100 Subject: [PATCH 047/152] Modifying the test for Futures.reduce --- akka-docs/java/code/akka/docs/future/FutureDocTestBase.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java b/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java index c3278f23bd..1e4e7fd350 100644 --- a/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java +++ b/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java @@ -267,13 +267,13 @@ public class FutureDocTestBase { //A sequence of Futures, in this case Strings Iterable> futures = source; - Future resultFuture = reduce(futures, new Function2() { - public String apply(String r, String t) { + Future resultFuture = reduce(futures, new Function2() { + public Object apply(Object r, String t) { return r + t; //Just concatenate } }, system.dispatcher()); - String result = Await.result(resultFuture, Duration.create(1, SECONDS)); + Object result = Await.result(resultFuture, Duration.create(1, SECONDS)); //#reduce assertEquals("ab", result); From 76521793a8b3b879341867906deefe0431e8d614 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 19 Jan 2012 18:08:13 +0100 Subject: [PATCH 048/152] #1690 - upgrading Netty to 3.3.0-Final --- project/AkkaBuild.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 091346de34..0ead5c3a8e 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -447,7 +447,7 @@ object Dependency { val Jersey = "1.3" val Jetty = "7.4.0.v20110414" val Logback = "0.9.28" - val Netty = "3.2.5.Final" + val Netty = "3.3.0.Final" val Protobuf = "2.4.1" val Rabbit = "2.3.1" val ScalaStm = "0.4" @@ -474,7 +474,7 @@ object Dependency { val jettyServlet = "org.eclipse.jetty" % "jetty-servlet" % V.Jetty // Eclipse license val log4j = "log4j" % "log4j" % "1.2.14" // ApacheV2 val mongoAsync = "com.mongodb.async" % "mongo-driver_2.9.0-1" % "0.2.9-1" // ApacheV2 - val netty = "org.jboss.netty" % "netty" % V.Netty // ApacheV2 + val netty = "io.netty" % "netty" % V.Netty // ApacheV2 val osgi = "org.osgi" % "org.osgi.core" % "4.2.0" // ApacheV2 val protobuf = "com.google.protobuf" % "protobuf-java" % V.Protobuf // New BSD val rabbit = "com.rabbitmq" % "amqp-client" % V.Rabbit // Mozilla Public License From 149fbea5a7bef924834d7f21d06c88a75eb0f00b Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 19 Jan 2012 18:21:06 +0100 Subject: [PATCH 049/152] Updating Copyright as per #1660 --- akka-actor-tests/src/test/java/akka/actor/JavaExtension.java | 2 +- akka-actor-tests/src/test/java/akka/util/JavaDuration.java | 2 +- akka-actor-tests/src/test/scala/akka/Messages.scala | 2 +- .../test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala | 2 +- .../src/test/scala/akka/actor/ActorLookupSpec.scala | 2 +- akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala | 2 +- .../src/test/scala/akka/actor/ActorSystemSpec.scala | 2 +- .../src/test/scala/akka/actor/ActorTimeoutSpec.scala | 2 +- akka-actor-tests/src/test/scala/akka/actor/DeployerSpec.scala | 2 +- akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala | 2 +- akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala | 2 +- .../src/test/scala/akka/actor/FSMTransitionSpec.scala | 2 +- .../src/test/scala/akka/actor/ForwardActorSpec.scala | 2 +- akka-actor-tests/src/test/scala/akka/actor/HotSwapSpec.scala | 2 +- akka-actor-tests/src/test/scala/akka/actor/JavaAPISpec.scala | 2 +- .../src/test/scala/akka/actor/LocalActorRefProviderSpec.scala | 2 +- .../src/test/scala/akka/actor/ReceiveTimeoutSpec.scala | 2 +- .../src/test/scala/akka/actor/RestartStrategySpec.scala | 2 +- akka-actor-tests/src/test/scala/akka/actor/Supervisor.scala | 2 +- .../src/test/scala/akka/actor/SupervisorHierarchySpec.scala | 2 +- .../src/test/scala/akka/actor/SupervisorMiscSpec.scala | 2 +- akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala | 2 +- .../src/test/scala/akka/actor/SupervisorTreeSpec.scala | 2 +- akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala | 2 +- akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala | 2 +- .../src/test/scala/akka/actor/dispatch/ActorModelSpec.scala | 2 +- .../src/test/scala/akka/actor/dispatch/DispatchersSpec.scala | 2 +- akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala | 2 +- .../src/test/scala/akka/dataflow/Future2Actor.scala | 2 +- .../src/test/scala/akka/event/EventStreamSpec.scala | 2 +- .../src/test/scala/akka/event/LoggingReceiveSpec.scala | 2 +- akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala | 2 +- akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala | 2 +- .../src/test/scala/akka/serialization/SerializeSpec.scala | 2 +- .../scala/akka/testkit/CallingThreadDispatcherModelSpec.scala | 2 +- akka-actor-tests/src/test/scala/akka/util/DurationSpec.scala | 2 +- akka-actor-tests/src/test/scala/akka/util/IndexSpec.scala | 2 +- .../src/test/scala/akka/util/JavaDurationSpec.scala | 2 +- akka-actor/src/main/java/akka/actor/Actors.java | 2 +- akka-actor/src/main/java/akka/dispatch/AbstractMailbox.java | 2 +- .../src/main/java/akka/dispatch/AbstractMessageDispatcher.java | 2 +- akka-actor/src/main/java/akka/dispatch/AbstractPromise.java | 2 +- akka-actor/src/main/scala/akka/AkkaException.scala | 2 +- akka-actor/src/main/scala/akka/actor/Actor.scala | 2 +- akka-actor/src/main/scala/akka/actor/ActorCell.scala | 2 +- akka-actor/src/main/scala/akka/actor/ActorPath.scala | 2 +- akka-actor/src/main/scala/akka/actor/ActorRef.scala | 2 +- akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala | 2 +- akka-actor/src/main/scala/akka/actor/ActorSelection.scala | 2 +- akka-actor/src/main/scala/akka/actor/ActorSystem.scala | 2 +- akka-actor/src/main/scala/akka/actor/Address.scala | 2 +- akka-actor/src/main/scala/akka/actor/Deployer.scala | 2 +- akka-actor/src/main/scala/akka/actor/Extension.scala | 2 +- akka-actor/src/main/scala/akka/actor/FSM.scala | 2 +- akka-actor/src/main/scala/akka/actor/FaultHandling.scala | 2 +- akka-actor/src/main/scala/akka/actor/Locker.scala | 2 +- akka-actor/src/main/scala/akka/actor/Props.scala | 2 +- akka-actor/src/main/scala/akka/actor/TypedActor.scala | 2 +- akka-actor/src/main/scala/akka/actor/UntypedActor.scala | 2 +- akka-actor/src/main/scala/akka/actor/package.scala | 2 +- .../src/main/scala/akka/config/ConfigurationException.scala | 2 +- .../src/main/scala/akka/dispatch/AbstractDispatcher.scala | 2 +- .../src/main/scala/akka/dispatch/BalancingDispatcher.scala | 2 +- akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala | 2 +- akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala | 2 +- akka-actor/src/main/scala/akka/dispatch/Future.scala | 2 +- akka-actor/src/main/scala/akka/dispatch/Mailbox.scala | 2 +- akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala | 2 +- akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala | 2 +- akka-actor/src/main/scala/akka/dispatch/japi/Future.scala | 2 +- akka-actor/src/main/scala/akka/event/DeathWatch.scala | 2 +- akka-actor/src/main/scala/akka/event/EventBus.scala | 2 +- akka-actor/src/main/scala/akka/event/EventStream.scala | 2 +- akka-actor/src/main/scala/akka/event/Logging.scala | 2 +- akka-actor/src/main/scala/akka/event/LoggingReceive.scala | 2 +- akka-actor/src/main/scala/akka/japi/JavaAPI.scala | 2 +- akka-actor/src/main/scala/akka/routing/ConnectionManager.scala | 2 +- akka-actor/src/main/scala/akka/routing/ConsistentHash.scala | 2 +- akka-actor/src/main/scala/akka/routing/Listeners.scala | 2 +- akka-actor/src/main/scala/akka/routing/Routing.scala | 2 +- akka-actor/src/main/scala/akka/routing/package.scala | 2 +- akka-actor/src/main/scala/akka/serialization/Format.scala | 2 +- .../src/main/scala/akka/serialization/Serialization.scala | 2 +- .../main/scala/akka/serialization/SerializationExtension.scala | 2 +- akka-actor/src/main/scala/akka/serialization/Serializer.scala | 2 +- akka-actor/src/main/scala/akka/util/BoundedBlockingQueue.scala | 2 +- akka-actor/src/main/scala/akka/util/BoxedType.scala | 2 +- .../src/main/scala/akka/util/ClassLoaderObjectInputStream.scala | 2 +- akka-actor/src/main/scala/akka/util/Convert.scala | 2 +- akka-actor/src/main/scala/akka/util/Crypt.scala | 2 +- akka-actor/src/main/scala/akka/util/Duration.scala | 2 +- akka-actor/src/main/scala/akka/util/HashCode.scala | 2 +- akka-actor/src/main/scala/akka/util/Helpers.scala | 2 +- akka-actor/src/main/scala/akka/util/Index.scala | 2 +- akka-actor/src/main/scala/akka/util/JMX.scala | 2 +- akka-actor/src/main/scala/akka/util/LockUtil.scala | 2 +- akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala | 2 +- akka-actor/src/main/scala/akka/util/SubclassifiedIndex.scala | 2 +- akka-actor/src/main/scala/akka/util/Unsafe.java | 2 +- akka-actor/src/main/scala/akka/util/cps/package.scala | 2 +- akka-actor/src/main/scala/akka/util/duration/package.scala | 2 +- akka-agent/src/main/scala/akka/agent/Agent.scala | 2 +- akka-camel-typed/src/main/scala/akka/camel/TypedCamel.scala | 2 +- akka-camel-typed/src/main/scala/akka/camel/TypedConsumer.scala | 2 +- .../src/main/scala/akka/camel/TypedConsumerPublisher.scala | 2 +- akka-camel/src/main/scala/akka/camel/TypedCamelAccess.scala | 2 +- .../src/main/java/akka/cluster/zookeeper/ZooKeeperQueue.java | 2 +- akka-cluster/src/main/scala/akka/cluster/BookKeeperServer.scala | 2 +- akka-cluster/src/main/scala/akka/cluster/Cluster.scala | 2 +- akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala | 2 +- akka-cluster/src/main/scala/akka/cluster/ClusterDeployer.scala | 2 +- akka-cluster/src/main/scala/akka/cluster/LocalCluster.scala | 2 +- akka-cluster/src/main/scala/akka/cluster/TransactionLog.scala | 2 +- .../scala/akka/cluster/metrics/LocalNodeMetricsManager.scala | 2 +- .../src/main/scala/akka/cluster/metrics/MetricsProvider.scala | 2 +- akka-cluster/src/main/scala/akka/cluster/storage/Storage.scala | 2 +- .../src/main/scala/akka/cluster/zookeeper/AkkaZkClient.scala | 2 +- .../src/main/scala/akka/cluster/zookeeper/AkkaZooKeeper.scala | 2 +- .../main/scala/akka/cluster/zookeeper/ZooKeeperBarrier.scala | 2 +- .../newleader/NewLeaderChangeListenerMultiJvmSpec.scala | 2 +- .../nodeconnected/NodeConnectedChangeListenerMultiJvmSpec.scala | 2 +- .../NodeDisconnectedChangeListenerMultiJvmSpec.scala | 2 +- .../api/configuration/ConfigurationStorageMultiJvmSpec.scala | 2 +- .../api/leader/election/LeaderElectionMultiJvmSpec.scala | 2 +- .../akka/cluster/api/registry/RegistryStoreMultiJvmSpec.scala | 2 +- .../scala/akka/cluster/deployment/DeploymentMultiJvmSpec.scala | 2 +- .../akka/cluster/metrics/local/LocalMetricsMultiJvmSpec.scala | 2 +- .../akka/cluster/metrics/remote/RemoteMetricsMultiJvmSpec.scala | 2 +- .../akka/cluster/migration/MigrationExplicitMultiJvmSpec.scala | 2 +- .../cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala | 2 +- ...icationTransactionLogWriteBehindNoSnapshotMultiJvmSpec.scala | 2 +- ...plicationTransactionLogWriteBehindSnapshotMultiJvmSpec.scala | 2 +- ...cationTransactionLogWriteThroughNoSnapshotMultiJvmSpec.scala | 2 +- ...licationTransactionLogWriteThroughSnapshotMultiJvmSpec.scala | 2 +- .../random/replicationfactor_1/Random1ReplicaMultiJvmSpec.scala | 2 +- .../replicationfactor_3/Random3ReplicasMultiJvmSpec.scala | 2 +- .../replicationfactor_1/RoundRobin1ReplicaMultiJvmSpec.scala | 2 +- .../replicationfactor_2/RoundRobin2ReplicasMultiJvmSpec.scala | 2 +- .../replicationfactor_3/RoundRobin3ReplicasMultiJvmSpec.scala | 2 +- .../scala/akka/cluster/sample/PingPongMultiJvmExample.scala | 2 +- .../scala/akka/cluster/AsynchronousTransactionLogSpec.scala | 2 +- .../test/scala/akka/cluster/SynchronousTransactionLogSpec.scala | 2 +- .../scala/akka/cluster/sample/ClusteredPingPongSample.scala | 2 +- .../src/test/scala/akka/cluster/sample/ComputeGridSample.scala | 2 +- akka-docs/general/code/akka/docs/config/ConfigDocSpec.scala | 2 +- akka-docs/java/code/akka/docs/actor/FaultHandlingTest.scala | 2 +- akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java | 2 +- akka-docs/java/code/akka/docs/actor/FirstUntypedActor.java | 2 +- akka-docs/java/code/akka/docs/actor/ImmutableMessage.java | 2 +- .../code/akka/docs/actor/MyReceivedTimeoutUntypedActor.java | 2 +- akka-docs/java/code/akka/docs/actor/MyUntypedActor.java | 2 +- akka-docs/java/code/akka/docs/actor/SchedulerDocTest.scala | 2 +- akka-docs/java/code/akka/docs/actor/SchedulerDocTestBase.java | 2 +- akka-docs/java/code/akka/docs/actor/TypedActorDocTest.scala | 2 +- akka-docs/java/code/akka/docs/actor/TypedActorDocTestBase.java | 2 +- akka-docs/java/code/akka/docs/actor/UntypedActorDocTest.scala | 2 +- .../java/code/akka/docs/actor/UntypedActorDocTestBase.java | 2 +- akka-docs/java/code/akka/docs/actor/UntypedActorSwapper.java | 2 +- akka-docs/java/code/akka/docs/agent/AgentDocJavaSpec.scala | 2 +- akka-docs/java/code/akka/docs/agent/AgentDocTest.java | 2 +- .../java/code/akka/docs/dispatcher/DispatcherDocTest.scala | 2 +- .../java/code/akka/docs/dispatcher/DispatcherDocTestBase.java | 2 +- akka-docs/java/code/akka/docs/event/LoggingDocTest.scala | 2 +- akka-docs/java/code/akka/docs/event/LoggingDocTestBase.java | 2 +- akka-docs/java/code/akka/docs/extension/ExtensionDocTest.scala | 2 +- .../java/code/akka/docs/extension/ExtensionDocTestBase.java | 2 +- akka-docs/java/code/akka/docs/future/FutureDocTest.scala | 2 +- akka-docs/java/code/akka/docs/future/FutureDocTestBase.java | 2 +- akka-docs/java/code/akka/docs/remoting/RemoteActorExample.java | 2 +- .../code/akka/docs/serialization/SerializationDocTest.scala | 2 +- .../code/akka/docs/serialization/SerializationDocTestBase.java | 2 +- .../java/code/akka/docs/transactor/CoordinatedCounter.java | 2 +- akka-docs/java/code/akka/docs/transactor/Coordinator.java | 2 +- akka-docs/java/code/akka/docs/transactor/Counter.java | 2 +- akka-docs/java/code/akka/docs/transactor/FriendlyCounter.java | 2 +- akka-docs/java/code/akka/docs/transactor/Increment.java | 2 +- akka-docs/java/code/akka/docs/transactor/Message.java | 2 +- .../java/code/akka/docs/transactor/TransactorDocJavaSpec.scala | 2 +- akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java | 2 +- .../code/akka/docs/actor/mailbox/DurableMailboxDocSpec.scala | 2 +- .../code/akka/docs/actor/mailbox/DurableMailboxDocTest.scala | 2 +- .../code/akka/docs/actor/mailbox/DurableMailboxDocTestBase.java | 2 +- akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala | 2 +- akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala | 2 +- akka-docs/scala/code/akka/docs/actor/SchedulerDocSpec.scala | 2 +- akka-docs/scala/code/akka/docs/actor/TypedActorDocSpec.scala | 2 +- akka-docs/scala/code/akka/docs/actor/UnnestedReceives.scala | 2 +- akka-docs/scala/code/akka/docs/agent/AgentDocSpec.scala | 2 +- .../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala | 2 +- akka-docs/scala/code/akka/docs/event/LoggingDocSpec.scala | 2 +- akka-docs/scala/code/akka/docs/extension/ExtensionDocSpec.scala | 2 +- akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala | 2 +- akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala | 2 +- .../scala/code/akka/docs/routing/RouterViaConfigExample.scala | 2 +- .../scala/code/akka/docs/routing/RouterViaProgramExample.scala | 2 +- .../code/akka/docs/serialization/SerializationDocSpec.scala | 2 +- akka-docs/scala/code/akka/docs/testkit/PlainWordSpec.scala | 2 +- akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala | 2 +- .../scala/code/akka/docs/transactor/TransactorDocSpec.scala | 2 +- .../main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala | 2 +- .../akka/actor/mailbox/BeanstalkBasedMailboxExtension.scala | 2 +- .../scala/akka/actor/mailbox/FileBasedMailboxExtension.scala | 2 +- .../src/main/scala/akka/actor/mailbox/FiledBasedMailbox.scala | 2 +- .../src/main/protocol/MailboxProtocol.proto | 2 +- .../src/main/scala/akka/actor/mailbox/DurableMailbox.scala | 2 +- .../src/test/scala/akka/actor/mailbox/DurableMailboxSpec.scala | 2 +- .../src/main/scala/akka/actor/mailbox/BSONSerialization.scala | 2 +- .../src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala | 2 +- .../scala/akka/actor/mailbox/MongoBasedMailboxExtension.scala | 2 +- .../src/main/scala/akka/actor/mailbox/MongoDurableMessage.scala | 2 +- .../src/main/scala/akka/actor/mailbox/RedisBasedMailbox.scala | 2 +- .../scala/akka/actor/mailbox/RedisBasedMailboxExtension.scala | 2 +- .../src/main/java/akka/cluster/zookeeper/ZooKeeperQueue.java | 2 +- .../main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala | 2 +- .../akka/actor/mailbox/ZooKeeperBasedMailboxExtension.scala | 2 +- .../src/main/scala/akka/cluster/zookeeper/AkkaZkClient.scala | 2 +- .../src/main/scala/akka/cluster/zookeeper/AkkaZooKeeper.scala | 2 +- akka-remote/src/main/protocol/RemoteProtocol.proto | 2 +- akka-remote/src/main/scala/akka/package.scala | 2 +- .../src/main/scala/akka/remote/AccrualFailureDetector.scala | 2 +- akka-remote/src/main/scala/akka/remote/Gossiper.scala | 2 +- akka-remote/src/main/scala/akka/remote/MessageSerializer.scala | 2 +- akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala | 2 +- akka-remote/src/main/scala/akka/remote/Remote.scala | 2 +- .../src/main/scala/akka/remote/RemoteActorRefProvider.scala | 2 +- .../src/main/scala/akka/remote/RemoteConnectionManager.scala | 2 +- akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala | 2 +- akka-remote/src/main/scala/akka/remote/RemoteSettings.scala | 2 +- akka-remote/src/main/scala/akka/remote/VectorClock.scala | 2 +- .../src/main/scala/akka/remote/netty/NettyRemoteSupport.scala | 2 +- .../src/main/scala/akka/routing/RemoteRouterConfig.scala | 2 +- akka-remote/src/main/scala/akka/serialization/Compression.scala | 2 +- .../src/main/scala/akka/serialization/ProtobufSerializer.scala | 2 +- .../src/multi-jvm/scala/akka/remote/AkkaRemoteSpec.scala | 2 +- akka-remote/src/multi-jvm/scala/akka/remote/Barrier.scala | 2 +- .../src/multi-jvm/scala/akka/remote/FileBasedBarrier.scala | 2 +- akka-remote/src/multi-jvm/scala/akka/remote/MultiJvmSync.scala | 2 +- akka-remote/src/multi-jvm/scala/akka/remote/QuietReporter.scala | 2 +- akka-remote/src/test/protocol/ProtobufProtocol.proto | 2 +- akka-remote/src/test/scala/akka/remote/NetworkFailureSpec.scala | 2 +- .../src/test/scala/akka/remote/RemoteCommunicationSpec.scala | 2 +- akka-remote/src/test/scala/akka/remote/RemoteDeployerSpec.scala | 2 +- akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala | 2 +- .../src/main/scala/sample/kernel/hello/HelloKernel.scala | 2 +- .../akka-sample-hello/src/main/scala/sample/hello/Main.scala | 2 +- .../main/java/sample/remote/calculator/java/InternalMsg.java | 2 +- .../sample/remote/calculator/java/JAdvancedCalculatorActor.java | 2 +- .../src/main/java/sample/remote/calculator/java/JCalcApp.java | 2 +- .../sample/remote/calculator/java/JCalculatorApplication.java | 2 +- .../main/java/sample/remote/calculator/java/JCreationActor.java | 2 +- .../main/java/sample/remote/calculator/java/JCreationApp.java | 2 +- .../sample/remote/calculator/java/JCreationApplication.java | 2 +- .../main/java/sample/remote/calculator/java/JLookupActor.java | 2 +- .../src/main/java/sample/remote/calculator/java/JLookupApp.java | 2 +- .../java/sample/remote/calculator/java/JLookupApplication.java | 2 +- .../sample/remote/calculator/java/JSimpleCalculatorActor.java | 2 +- .../src/main/java/sample/remote/calculator/java/Op.java | 2 +- .../scala/sample/remote/calculator/CalculatorApplication.scala | 2 +- .../scala/sample/remote/calculator/CreationApplication.scala | 2 +- .../main/scala/sample/remote/calculator/LookupApplication.scala | 2 +- .../src/main/scala/sample/remote/calculator/MathOp.scala | 2 +- akka-slf4j/src/main/scala/akka/event/slf4j/SLF4J.scala | 2 +- .../src/main/scala/akka/testkit/CallingThreadDispatcher.scala | 2 +- akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala | 2 +- akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala | 2 +- .../src/main/scala/akka/testkit/TestEventListener.scala | 2 +- akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala | 2 +- akka-testkit/src/main/scala/akka/testkit/TestKit.scala | 2 +- akka-testkit/src/main/scala/akka/testkit/TestKitExtension.scala | 2 +- akka-testkit/src/main/scala/akka/testkit/TestLatch.scala | 2 +- akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala | 2 +- akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala | 2 +- akka-testkit/src/test/scala/akka/testkit/TestFSMRefSpec.scala | 2 +- akka-transactor/src/main/scala/akka/transactor/Atomically.scala | 2 +- .../src/main/scala/akka/transactor/Coordinated.scala | 2 +- akka-transactor/src/main/scala/akka/transactor/Transactor.scala | 2 +- .../src/main/scala/akka/transactor/UntypedTransactor.scala | 2 +- .../src/test/java/akka/transactor/ExpectedFailureException.java | 2 +- akka-transactor/src/test/java/akka/transactor/Increment.java | 2 +- .../test/java/akka/transactor/UntypedCoordinatedCounter.java | 2 +- .../java/akka/transactor/UntypedCoordinatedIncrementTest.java | 2 +- .../src/test/java/akka/transactor/UntypedCounter.java | 2 +- .../src/test/java/akka/transactor/UntypedFailer.java | 2 +- .../src/test/java/akka/transactor/UntypedTransactorTest.java | 2 +- .../test/scala/akka/transactor/CoordinatedIncrementSpec.scala | 2 +- .../src/test/scala/akka/transactor/FickleFriendsSpec.scala | 2 +- .../test/scala/akka/transactor/JavaUntypedCoordinatedSpec.scala | 2 +- .../test/scala/akka/transactor/JavaUntypedTransactorSpec.scala | 2 +- .../src/test/scala/akka/transactor/TransactorSpec.scala | 2 +- .../src/main/java/akka/tutorial/first/java/Pi.java | 2 +- akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala | 2 +- .../akka-tutorial-first/src/test/scala/WorkerSpec.scala | 2 +- project/AkkaBuild.scala | 2 +- scripts/generate_config_with_secure_cookie.sh | 2 +- 294 files changed, 294 insertions(+), 294 deletions(-) diff --git a/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java b/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java index e7597309c4..dfb3e9230e 100644 --- a/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java +++ b/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor; diff --git a/akka-actor-tests/src/test/java/akka/util/JavaDuration.java b/akka-actor-tests/src/test/java/akka/util/JavaDuration.java index 5a833f5b51..835a0301c7 100644 --- a/akka-actor-tests/src/test/java/akka/util/JavaDuration.java +++ b/akka-actor-tests/src/test/java/akka/util/JavaDuration.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.util; diff --git a/akka-actor-tests/src/test/scala/akka/Messages.scala b/akka-actor-tests/src/test/scala/akka/Messages.scala index c6f2116509..32f80c7b24 100644 --- a/akka-actor-tests/src/test/scala/akka/Messages.scala +++ b/akka-actor-tests/src/test/scala/akka/Messages.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala index c059497259..13ccddefe2 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala index da6115ef5f..38695f8258 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala index ceea20b2c7..63f8ca242e 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala index 6f8c364ff8..3ddb616982 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala index ceb7bd0783..34e073cd2f 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor-tests/src/test/scala/akka/actor/DeployerSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/DeployerSpec.scala index 89a153ad57..f2443d6edb 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/DeployerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/DeployerSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala index 67c94d3dd7..ca6d90e721 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala index 7168daa265..bf5a1974ee 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala index 9db408770c..cc63b76704 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala index 12b0c796f6..aaa19b4561 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor-tests/src/test/scala/akka/actor/HotSwapSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/HotSwapSpec.scala index e3027a4c00..236d3bd014 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/HotSwapSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/HotSwapSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor-tests/src/test/scala/akka/actor/JavaAPISpec.scala b/akka-actor-tests/src/test/scala/akka/actor/JavaAPISpec.scala index 49fcc6d638..ca7210beaf 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/JavaAPISpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/JavaAPISpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor-tests/src/test/scala/akka/actor/LocalActorRefProviderSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/LocalActorRefProviderSpec.scala index e55eea31ff..82cd08fa77 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/LocalActorRefProviderSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/LocalActorRefProviderSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor-tests/src/test/scala/akka/actor/ReceiveTimeoutSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ReceiveTimeoutSpec.scala index 2671fa9b9a..0a28a8597e 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ReceiveTimeoutSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ReceiveTimeoutSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala index b627046052..e2fd22a030 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor-tests/src/test/scala/akka/actor/Supervisor.scala b/akka-actor-tests/src/test/scala/akka/actor/Supervisor.scala index 174939915d..0bf8183137 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/Supervisor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/Supervisor.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala index fb34e2345b..576c328bc7 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala index 653342c193..e269456e9b 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala index 9fe8ffb63a..30b1a24493 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala index 3985f6ea48..27ecec4863 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala b/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala index 3b5f29c950..73f3b416e9 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala index 2d99a4925f..17bbd948c3 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala @@ -1,7 +1,7 @@ package akka.actor /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach } diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala index fb75ab5593..71cf23ae7e 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor.dispatch diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatchersSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatchersSpec.scala index 4ebf0d0de8..d169981a5a 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatchersSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatchersSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor.dispatch diff --git a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala index 9b214423c3..191bbd101f 100644 --- a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.config diff --git a/akka-actor-tests/src/test/scala/akka/dataflow/Future2Actor.scala b/akka-actor-tests/src/test/scala/akka/dataflow/Future2Actor.scala index 5d24b9678f..c4668db428 100644 --- a/akka-actor-tests/src/test/scala/akka/dataflow/Future2Actor.scala +++ b/akka-actor-tests/src/test/scala/akka/dataflow/Future2Actor.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.dataflow diff --git a/akka-actor-tests/src/test/scala/akka/event/EventStreamSpec.scala b/akka-actor-tests/src/test/scala/akka/event/EventStreamSpec.scala index 9a41c80f6d..add8173085 100644 --- a/akka-actor-tests/src/test/scala/akka/event/EventStreamSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/event/EventStreamSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.event diff --git a/akka-actor-tests/src/test/scala/akka/event/LoggingReceiveSpec.scala b/akka-actor-tests/src/test/scala/akka/event/LoggingReceiveSpec.scala index bcfb9c391b..7e756657f5 100644 --- a/akka-actor-tests/src/test/scala/akka/event/LoggingReceiveSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/event/LoggingReceiveSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.event diff --git a/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala index 35cc429fa6..d2849188a8 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ResizerSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.routing diff --git a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala index 910985151d..bcf3e6328e 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.routing diff --git a/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala b/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala index 51dc26d6c5..bb86afb5eb 100644 --- a/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.serialization diff --git a/akka-actor-tests/src/test/scala/akka/testkit/CallingThreadDispatcherModelSpec.scala b/akka-actor-tests/src/test/scala/akka/testkit/CallingThreadDispatcherModelSpec.scala index 800148a187..ab0f7ec6eb 100644 --- a/akka-actor-tests/src/test/scala/akka/testkit/CallingThreadDispatcherModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/testkit/CallingThreadDispatcherModelSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.testkit diff --git a/akka-actor-tests/src/test/scala/akka/util/DurationSpec.scala b/akka-actor-tests/src/test/scala/akka/util/DurationSpec.scala index 4a04a648bf..dd2867019c 100644 --- a/akka-actor-tests/src/test/scala/akka/util/DurationSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/util/DurationSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.util diff --git a/akka-actor-tests/src/test/scala/akka/util/IndexSpec.scala b/akka-actor-tests/src/test/scala/akka/util/IndexSpec.scala index 9f869fe907..4abe8c508d 100644 --- a/akka-actor-tests/src/test/scala/akka/util/IndexSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/util/IndexSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.util diff --git a/akka-actor-tests/src/test/scala/akka/util/JavaDurationSpec.scala b/akka-actor-tests/src/test/scala/akka/util/JavaDurationSpec.scala index aafbf3d133..2278873b56 100644 --- a/akka-actor-tests/src/test/scala/akka/util/JavaDurationSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/util/JavaDurationSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.util diff --git a/akka-actor/src/main/java/akka/actor/Actors.java b/akka-actor/src/main/java/akka/actor/Actors.java index dd8763aad0..f0ee734c3b 100644 --- a/akka-actor/src/main/java/akka/actor/Actors.java +++ b/akka-actor/src/main/java/akka/actor/Actors.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor; diff --git a/akka-actor/src/main/java/akka/dispatch/AbstractMailbox.java b/akka-actor/src/main/java/akka/dispatch/AbstractMailbox.java index dbe87482dc..c4ac378fad 100644 --- a/akka-actor/src/main/java/akka/dispatch/AbstractMailbox.java +++ b/akka-actor/src/main/java/akka/dispatch/AbstractMailbox.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.dispatch; diff --git a/akka-actor/src/main/java/akka/dispatch/AbstractMessageDispatcher.java b/akka-actor/src/main/java/akka/dispatch/AbstractMessageDispatcher.java index 5d4eee3931..8248fd9e07 100644 --- a/akka-actor/src/main/java/akka/dispatch/AbstractMessageDispatcher.java +++ b/akka-actor/src/main/java/akka/dispatch/AbstractMessageDispatcher.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.dispatch; diff --git a/akka-actor/src/main/java/akka/dispatch/AbstractPromise.java b/akka-actor/src/main/java/akka/dispatch/AbstractPromise.java index a770a0b27a..90344eb10a 100644 --- a/akka-actor/src/main/java/akka/dispatch/AbstractPromise.java +++ b/akka-actor/src/main/java/akka/dispatch/AbstractPromise.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.dispatch; diff --git a/akka-actor/src/main/scala/akka/AkkaException.scala b/akka-actor/src/main/scala/akka/AkkaException.scala index b8d83abf7a..eb4a1cea8e 100644 --- a/akka-actor/src/main/scala/akka/AkkaException.scala +++ b/akka-actor/src/main/scala/akka/AkkaException.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index d69696e0b5..2cdd41c36e 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 34b26629af..0e813e5cd2 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor/src/main/scala/akka/actor/ActorPath.scala b/akka-actor/src/main/scala/akka/actor/ActorPath.scala index 2aa48a093d..d0f0d8154b 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorPath.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorPath.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor import scala.annotation.tailrec diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 94ec966468..c623e438de 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 7bc9f502d1..5073725fa6 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor/src/main/scala/akka/actor/ActorSelection.scala b/akka-actor/src/main/scala/akka/actor/ActorSelection.scala index fcb7db7167..b407868270 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSelection.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSelection.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor import java.util.regex.Pattern diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index a84d080536..07ed860116 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor/src/main/scala/akka/actor/Address.scala b/akka-actor/src/main/scala/akka/actor/Address.scala index e9097d72d6..6cd6c369dd 100644 --- a/akka-actor/src/main/scala/akka/actor/Address.scala +++ b/akka-actor/src/main/scala/akka/actor/Address.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor import java.net.URI diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index 54ba5c1a57..a13d3a8424 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor/src/main/scala/akka/actor/Extension.scala b/akka-actor/src/main/scala/akka/actor/Extension.scala index 23439b263d..1850661417 100644 --- a/akka-actor/src/main/scala/akka/actor/Extension.scala +++ b/akka-actor/src/main/scala/akka/actor/Extension.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor/src/main/scala/akka/actor/FSM.scala b/akka-actor/src/main/scala/akka/actor/FSM.scala index eb573df767..bf20b43274 100644 --- a/akka-actor/src/main/scala/akka/actor/FSM.scala +++ b/akka-actor/src/main/scala/akka/actor/FSM.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala index 2b8e23c6fb..8dc816f8ad 100644 --- a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor/src/main/scala/akka/actor/Locker.scala b/akka-actor/src/main/scala/akka/actor/Locker.scala index d4fd1badd5..0aef9e6850 100644 --- a/akka-actor/src/main/scala/akka/actor/Locker.scala +++ b/akka-actor/src/main/scala/akka/actor/Locker.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor/src/main/scala/akka/actor/Props.scala b/akka-actor/src/main/scala/akka/actor/Props.scala index dccfd0bd3c..8cce3f35e5 100644 --- a/akka-actor/src/main/scala/akka/actor/Props.scala +++ b/akka-actor/src/main/scala/akka/actor/Props.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index b3913b8138..15397ab966 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -1,7 +1,7 @@ package akka.actor /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ import akka.japi.{ Creator, Option ⇒ JOption } diff --git a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala index c5822db7e9..76af2cfb4e 100644 --- a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor diff --git a/akka-actor/src/main/scala/akka/actor/package.scala b/akka-actor/src/main/scala/akka/actor/package.scala index cfe5bc1b0d..c03d7f8689 100644 --- a/akka-actor/src/main/scala/akka/actor/package.scala +++ b/akka-actor/src/main/scala/akka/actor/package.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka diff --git a/akka-actor/src/main/scala/akka/config/ConfigurationException.scala b/akka-actor/src/main/scala/akka/config/ConfigurationException.scala index c2bad21ac6..ba0a3a2234 100644 --- a/akka-actor/src/main/scala/akka/config/ConfigurationException.scala +++ b/akka-actor/src/main/scala/akka/config/ConfigurationException.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.config diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index 66014e25cc..812cda9f9f 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.dispatch diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index 31ca64867c..92c5d1e967 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.dispatch diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala index 2511dbc8e2..cc60decec9 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.dispatch diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala index aa1907b093..3dfffcdbbf 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.dispatch diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 35eb49e24d..72c59f5e14 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -1,6 +1,6 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.dispatch diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index d88887d3db..dacf39593d 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.dispatch diff --git a/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala index 8e8f229c90..d89df72188 100644 --- a/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.dispatch diff --git a/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala index 77669cfd1a..e073e18b66 100644 --- a/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala +++ b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.dispatch diff --git a/akka-actor/src/main/scala/akka/dispatch/japi/Future.scala b/akka-actor/src/main/scala/akka/dispatch/japi/Future.scala index 64383df243..ac4ef7694e 100644 --- a/akka-actor/src/main/scala/akka/dispatch/japi/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/japi/Future.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.dispatch.japi diff --git a/akka-actor/src/main/scala/akka/event/DeathWatch.scala b/akka-actor/src/main/scala/akka/event/DeathWatch.scala index b03dbe97b9..7469f6609f 100644 --- a/akka-actor/src/main/scala/akka/event/DeathWatch.scala +++ b/akka-actor/src/main/scala/akka/event/DeathWatch.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.event diff --git a/akka-actor/src/main/scala/akka/event/EventBus.scala b/akka-actor/src/main/scala/akka/event/EventBus.scala index bd0fa8a7ce..f79ee94a8c 100644 --- a/akka-actor/src/main/scala/akka/event/EventBus.scala +++ b/akka-actor/src/main/scala/akka/event/EventBus.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.event diff --git a/akka-actor/src/main/scala/akka/event/EventStream.scala b/akka-actor/src/main/scala/akka/event/EventStream.scala index 6ad2d0fbdf..27f0c71515 100644 --- a/akka-actor/src/main/scala/akka/event/EventStream.scala +++ b/akka-actor/src/main/scala/akka/event/EventStream.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.event diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index 07a1da1da5..a5b4f6923c 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.event diff --git a/akka-actor/src/main/scala/akka/event/LoggingReceive.scala b/akka-actor/src/main/scala/akka/event/LoggingReceive.scala index 27d829de5e..92a4166be8 100644 --- a/akka-actor/src/main/scala/akka/event/LoggingReceive.scala +++ b/akka-actor/src/main/scala/akka/event/LoggingReceive.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.event diff --git a/akka-actor/src/main/scala/akka/japi/JavaAPI.scala b/akka-actor/src/main/scala/akka/japi/JavaAPI.scala index 070a10b2a8..e414d0fee6 100644 --- a/akka-actor/src/main/scala/akka/japi/JavaAPI.scala +++ b/akka-actor/src/main/scala/akka/japi/JavaAPI.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.japi diff --git a/akka-actor/src/main/scala/akka/routing/ConnectionManager.scala b/akka-actor/src/main/scala/akka/routing/ConnectionManager.scala index 135546ad2b..3ce73f5020 100644 --- a/akka-actor/src/main/scala/akka/routing/ConnectionManager.scala +++ b/akka-actor/src/main/scala/akka/routing/ConnectionManager.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.routing diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala index ead70b4b7a..130db2be3e 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHash.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.routing diff --git a/akka-actor/src/main/scala/akka/routing/Listeners.scala b/akka-actor/src/main/scala/akka/routing/Listeners.scala index 05f22f8b4a..39fbf6355d 100644 --- a/akka-actor/src/main/scala/akka/routing/Listeners.scala +++ b/akka-actor/src/main/scala/akka/routing/Listeners.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.routing diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index 2bbb819b2d..7fdbd1db5a 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.routing diff --git a/akka-actor/src/main/scala/akka/routing/package.scala b/akka-actor/src/main/scala/akka/routing/package.scala index 44662856b1..c1672fc0e2 100644 --- a/akka-actor/src/main/scala/akka/routing/package.scala +++ b/akka-actor/src/main/scala/akka/routing/package.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka diff --git a/akka-actor/src/main/scala/akka/serialization/Format.scala b/akka-actor/src/main/scala/akka/serialization/Format.scala index 41f9e9ce73..41a8eed658 100644 --- a/akka-actor/src/main/scala/akka/serialization/Format.scala +++ b/akka-actor/src/main/scala/akka/serialization/Format.scala @@ -1,7 +1,7 @@ package akka.serialization /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ import akka.actor.Actor diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index 077ff12e1a..eaa0ce0925 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.serialization diff --git a/akka-actor/src/main/scala/akka/serialization/SerializationExtension.scala b/akka-actor/src/main/scala/akka/serialization/SerializationExtension.scala index 6bc47ecf41..9d9815f412 100644 --- a/akka-actor/src/main/scala/akka/serialization/SerializationExtension.scala +++ b/akka-actor/src/main/scala/akka/serialization/SerializationExtension.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.serialization diff --git a/akka-actor/src/main/scala/akka/serialization/Serializer.scala b/akka-actor/src/main/scala/akka/serialization/Serializer.scala index a39f77b210..ffe7f50de9 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serializer.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serializer.scala @@ -1,7 +1,7 @@ package akka.serialization /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ import java.io.{ ObjectOutputStream, ByteArrayOutputStream, ObjectInputStream, ByteArrayInputStream } diff --git a/akka-actor/src/main/scala/akka/util/BoundedBlockingQueue.scala b/akka-actor/src/main/scala/akka/util/BoundedBlockingQueue.scala index fb93f4a639..cf09a8ecdd 100644 --- a/akka-actor/src/main/scala/akka/util/BoundedBlockingQueue.scala +++ b/akka-actor/src/main/scala/akka/util/BoundedBlockingQueue.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.util diff --git a/akka-actor/src/main/scala/akka/util/BoxedType.scala b/akka-actor/src/main/scala/akka/util/BoxedType.scala index 4051dc6a51..d2c5092be4 100644 --- a/akka-actor/src/main/scala/akka/util/BoxedType.scala +++ b/akka-actor/src/main/scala/akka/util/BoxedType.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.util diff --git a/akka-actor/src/main/scala/akka/util/ClassLoaderObjectInputStream.scala b/akka-actor/src/main/scala/akka/util/ClassLoaderObjectInputStream.scala index 07bf054eaa..61f07d2b8a 100644 --- a/akka-actor/src/main/scala/akka/util/ClassLoaderObjectInputStream.scala +++ b/akka-actor/src/main/scala/akka/util/ClassLoaderObjectInputStream.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.util diff --git a/akka-actor/src/main/scala/akka/util/Convert.scala b/akka-actor/src/main/scala/akka/util/Convert.scala index 42a98f3849..a805b17fb2 100644 --- a/akka-actor/src/main/scala/akka/util/Convert.scala +++ b/akka-actor/src/main/scala/akka/util/Convert.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.util diff --git a/akka-actor/src/main/scala/akka/util/Crypt.scala b/akka-actor/src/main/scala/akka/util/Crypt.scala index 50e8c881a6..7dd678e748 100644 --- a/akka-actor/src/main/scala/akka/util/Crypt.scala +++ b/akka-actor/src/main/scala/akka/util/Crypt.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.util diff --git a/akka-actor/src/main/scala/akka/util/Duration.scala b/akka-actor/src/main/scala/akka/util/Duration.scala index 1406ad8564..0442619c45 100644 --- a/akka-actor/src/main/scala/akka/util/Duration.scala +++ b/akka-actor/src/main/scala/akka/util/Duration.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.util diff --git a/akka-actor/src/main/scala/akka/util/HashCode.scala b/akka-actor/src/main/scala/akka/util/HashCode.scala index 40b740bfd6..9e19c5e7c6 100644 --- a/akka-actor/src/main/scala/akka/util/HashCode.scala +++ b/akka-actor/src/main/scala/akka/util/HashCode.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.util diff --git a/akka-actor/src/main/scala/akka/util/Helpers.scala b/akka-actor/src/main/scala/akka/util/Helpers.scala index a03715b4b8..d98e0a49f1 100644 --- a/akka-actor/src/main/scala/akka/util/Helpers.scala +++ b/akka-actor/src/main/scala/akka/util/Helpers.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.util diff --git a/akka-actor/src/main/scala/akka/util/Index.scala b/akka-actor/src/main/scala/akka/util/Index.scala index b7cb1a74a4..1153c9e045 100644 --- a/akka-actor/src/main/scala/akka/util/Index.scala +++ b/akka-actor/src/main/scala/akka/util/Index.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.util diff --git a/akka-actor/src/main/scala/akka/util/JMX.scala b/akka-actor/src/main/scala/akka/util/JMX.scala index 9a9f0530fb..44d1410d6b 100644 --- a/akka-actor/src/main/scala/akka/util/JMX.scala +++ b/akka-actor/src/main/scala/akka/util/JMX.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.util diff --git a/akka-actor/src/main/scala/akka/util/LockUtil.scala b/akka-actor/src/main/scala/akka/util/LockUtil.scala index 65bcf563fc..c3295d4b52 100644 --- a/akka-actor/src/main/scala/akka/util/LockUtil.scala +++ b/akka-actor/src/main/scala/akka/util/LockUtil.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.util diff --git a/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala b/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala index 70b6fa5a03..18ed6a5020 100644 --- a/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala +++ b/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.util diff --git a/akka-actor/src/main/scala/akka/util/SubclassifiedIndex.scala b/akka-actor/src/main/scala/akka/util/SubclassifiedIndex.scala index 7768a73fd2..4149a0b0b1 100644 --- a/akka-actor/src/main/scala/akka/util/SubclassifiedIndex.scala +++ b/akka-actor/src/main/scala/akka/util/SubclassifiedIndex.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.util diff --git a/akka-actor/src/main/scala/akka/util/Unsafe.java b/akka-actor/src/main/scala/akka/util/Unsafe.java index b1d7f61290..608cb3d46e 100644 --- a/akka-actor/src/main/scala/akka/util/Unsafe.java +++ b/akka-actor/src/main/scala/akka/util/Unsafe.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ diff --git a/akka-actor/src/main/scala/akka/util/cps/package.scala b/akka-actor/src/main/scala/akka/util/cps/package.scala index cf23f304a6..198c2beacd 100644 --- a/akka-actor/src/main/scala/akka/util/cps/package.scala +++ b/akka-actor/src/main/scala/akka/util/cps/package.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.util diff --git a/akka-actor/src/main/scala/akka/util/duration/package.scala b/akka-actor/src/main/scala/akka/util/duration/package.scala index 88a328d6d8..b052cca93a 100644 --- a/akka-actor/src/main/scala/akka/util/duration/package.scala +++ b/akka-actor/src/main/scala/akka/util/duration/package.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.util diff --git a/akka-agent/src/main/scala/akka/agent/Agent.scala b/akka-agent/src/main/scala/akka/agent/Agent.scala index dffd8df1cc..94f85268a3 100644 --- a/akka-agent/src/main/scala/akka/agent/Agent.scala +++ b/akka-agent/src/main/scala/akka/agent/Agent.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.agent diff --git a/akka-camel-typed/src/main/scala/akka/camel/TypedCamel.scala b/akka-camel-typed/src/main/scala/akka/camel/TypedCamel.scala index 67499bb02b..e2a6c0b35a 100644 --- a/akka-camel-typed/src/main/scala/akka/camel/TypedCamel.scala +++ b/akka-camel-typed/src/main/scala/akka/camel/TypedCamel.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.camel diff --git a/akka-camel-typed/src/main/scala/akka/camel/TypedConsumer.scala b/akka-camel-typed/src/main/scala/akka/camel/TypedConsumer.scala index 74d63401ed..c7b8059153 100644 --- a/akka-camel-typed/src/main/scala/akka/camel/TypedConsumer.scala +++ b/akka-camel-typed/src/main/scala/akka/camel/TypedConsumer.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.camel diff --git a/akka-camel-typed/src/main/scala/akka/camel/TypedConsumerPublisher.scala b/akka-camel-typed/src/main/scala/akka/camel/TypedConsumerPublisher.scala index 850cbd640b..6822665bb9 100644 --- a/akka-camel-typed/src/main/scala/akka/camel/TypedConsumerPublisher.scala +++ b/akka-camel-typed/src/main/scala/akka/camel/TypedConsumerPublisher.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.camel diff --git a/akka-camel/src/main/scala/akka/camel/TypedCamelAccess.scala b/akka-camel/src/main/scala/akka/camel/TypedCamelAccess.scala index 2e3730686e..876b65f66a 100644 --- a/akka-camel/src/main/scala/akka/camel/TypedCamelAccess.scala +++ b/akka-camel/src/main/scala/akka/camel/TypedCamelAccess.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.camel diff --git a/akka-cluster/src/main/java/akka/cluster/zookeeper/ZooKeeperQueue.java b/akka-cluster/src/main/java/akka/cluster/zookeeper/ZooKeeperQueue.java index af76896f4b..8867d97e00 100644 --- a/akka-cluster/src/main/java/akka/cluster/zookeeper/ZooKeeperQueue.java +++ b/akka-cluster/src/main/java/akka/cluster/zookeeper/ZooKeeperQueue.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.zookeeper; diff --git a/akka-cluster/src/main/scala/akka/cluster/BookKeeperServer.scala b/akka-cluster/src/main/scala/akka/cluster/BookKeeperServer.scala index 7c3b57969d..679af24d03 100644 --- a/akka-cluster/src/main/scala/akka/cluster/BookKeeperServer.scala +++ b/akka-cluster/src/main/scala/akka/cluster/BookKeeperServer.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster diff --git a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala index 505474213f..130149b491 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala index 84d23af736..29f56a5966 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterDeployer.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterDeployer.scala index 4cc791fd89..61a393360c 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterDeployer.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterDeployer.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster diff --git a/akka-cluster/src/main/scala/akka/cluster/LocalCluster.scala b/akka-cluster/src/main/scala/akka/cluster/LocalCluster.scala index ff78d05796..d8a0ac6027 100644 --- a/akka-cluster/src/main/scala/akka/cluster/LocalCluster.scala +++ b/akka-cluster/src/main/scala/akka/cluster/LocalCluster.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster diff --git a/akka-cluster/src/main/scala/akka/cluster/TransactionLog.scala b/akka-cluster/src/main/scala/akka/cluster/TransactionLog.scala index 8aa1727ca9..ce9eb300f5 100644 --- a/akka-cluster/src/main/scala/akka/cluster/TransactionLog.scala +++ b/akka-cluster/src/main/scala/akka/cluster/TransactionLog.scala @@ -1,7 +1,7 @@ package akka.cluster /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ import org.apache.bookkeeper.client.{ BookKeeper, LedgerHandle, LedgerEntry, BKException, AsyncCallback } diff --git a/akka-cluster/src/main/scala/akka/cluster/metrics/LocalNodeMetricsManager.scala b/akka-cluster/src/main/scala/akka/cluster/metrics/LocalNodeMetricsManager.scala index 3267671fba..c366ed598c 100644 --- a/akka-cluster/src/main/scala/akka/cluster/metrics/LocalNodeMetricsManager.scala +++ b/akka-cluster/src/main/scala/akka/cluster/metrics/LocalNodeMetricsManager.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.metrics diff --git a/akka-cluster/src/main/scala/akka/cluster/metrics/MetricsProvider.scala b/akka-cluster/src/main/scala/akka/cluster/metrics/MetricsProvider.scala index ba17d4c1c0..0b366ef9c8 100644 --- a/akka-cluster/src/main/scala/akka/cluster/metrics/MetricsProvider.scala +++ b/akka-cluster/src/main/scala/akka/cluster/metrics/MetricsProvider.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.metrics diff --git a/akka-cluster/src/main/scala/akka/cluster/storage/Storage.scala b/akka-cluster/src/main/scala/akka/cluster/storage/Storage.scala index a7a5cf0643..a402f2def1 100644 --- a/akka-cluster/src/main/scala/akka/cluster/storage/Storage.scala +++ b/akka-cluster/src/main/scala/akka/cluster/storage/Storage.scala @@ -1,7 +1,7 @@ package akka.cluster.storage /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ import akka.cluster.zookeeper.AkkaZkClient import akka.AkkaException diff --git a/akka-cluster/src/main/scala/akka/cluster/zookeeper/AkkaZkClient.scala b/akka-cluster/src/main/scala/akka/cluster/zookeeper/AkkaZkClient.scala index fd27d894bf..9137959877 100644 --- a/akka-cluster/src/main/scala/akka/cluster/zookeeper/AkkaZkClient.scala +++ b/akka-cluster/src/main/scala/akka/cluster/zookeeper/AkkaZkClient.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.zookeeper diff --git a/akka-cluster/src/main/scala/akka/cluster/zookeeper/AkkaZooKeeper.scala b/akka-cluster/src/main/scala/akka/cluster/zookeeper/AkkaZooKeeper.scala index ce9e74c260..b5165ffb72 100644 --- a/akka-cluster/src/main/scala/akka/cluster/zookeeper/AkkaZooKeeper.scala +++ b/akka-cluster/src/main/scala/akka/cluster/zookeeper/AkkaZooKeeper.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.zookeeper diff --git a/akka-cluster/src/main/scala/akka/cluster/zookeeper/ZooKeeperBarrier.scala b/akka-cluster/src/main/scala/akka/cluster/zookeeper/ZooKeeperBarrier.scala index d737403dac..c1f51ceb96 100644 --- a/akka-cluster/src/main/scala/akka/cluster/zookeeper/ZooKeeperBarrier.scala +++ b/akka-cluster/src/main/scala/akka/cluster/zookeeper/ZooKeeperBarrier.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.zookeeper diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/api/changelisteners/newleader/NewLeaderChangeListenerMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/api/changelisteners/newleader/NewLeaderChangeListenerMultiJvmSpec.scala index 546d68b3b9..f1b9f5a7ae 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/api/changelisteners/newleader/NewLeaderChangeListenerMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/api/changelisteners/newleader/NewLeaderChangeListenerMultiJvmSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.api.changelisteners.newleader diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/api/changelisteners/nodeconnected/NodeConnectedChangeListenerMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/api/changelisteners/nodeconnected/NodeConnectedChangeListenerMultiJvmSpec.scala index 6c118da2d3..deec5c19e6 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/api/changelisteners/nodeconnected/NodeConnectedChangeListenerMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/api/changelisteners/nodeconnected/NodeConnectedChangeListenerMultiJvmSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.api.changelisteners.nodeconnected diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/api/changelisteners/nodedisconnected/NodeDisconnectedChangeListenerMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/api/changelisteners/nodedisconnected/NodeDisconnectedChangeListenerMultiJvmSpec.scala index ab1202d118..54a327126e 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/api/changelisteners/nodedisconnected/NodeDisconnectedChangeListenerMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/api/changelisteners/nodedisconnected/NodeDisconnectedChangeListenerMultiJvmSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.api.changelisteners.nodedisconnected diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/api/configuration/ConfigurationStorageMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/api/configuration/ConfigurationStorageMultiJvmSpec.scala index d780a30776..f9aabbb004 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/api/configuration/ConfigurationStorageMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/api/configuration/ConfigurationStorageMultiJvmSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.api.configuration diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/api/leader/election/LeaderElectionMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/api/leader/election/LeaderElectionMultiJvmSpec.scala index ed8c4b4c5e..479f77e0d3 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/api/leader/election/LeaderElectionMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/api/leader/election/LeaderElectionMultiJvmSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.api.leader.election diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/api/registry/RegistryStoreMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/api/registry/RegistryStoreMultiJvmSpec.scala index f9bbac2b70..c20bf9269c 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/api/registry/RegistryStoreMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/api/registry/RegistryStoreMultiJvmSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.api.registry diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/deployment/DeploymentMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/deployment/DeploymentMultiJvmSpec.scala index 0d5a66253f..ef0b79b4a7 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/deployment/DeploymentMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/deployment/DeploymentMultiJvmSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.deployment diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/metrics/local/LocalMetricsMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/metrics/local/LocalMetricsMultiJvmSpec.scala index 50b7741758..380d68d8ef 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/metrics/local/LocalMetricsMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/metrics/local/LocalMetricsMultiJvmSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.metrics.local diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/metrics/remote/RemoteMetricsMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/metrics/remote/RemoteMetricsMultiJvmSpec.scala index 22054bc0d0..8c4730dc90 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/metrics/remote/RemoteMetricsMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/metrics/remote/RemoteMetricsMultiJvmSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.metrics.remote diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/migration/MigrationExplicitMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/migration/MigrationExplicitMultiJvmSpec.scala index 22333be938..7dfdec2f7c 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/migration/MigrationExplicitMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/migration/MigrationExplicitMultiJvmSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. * * * package akka.cluster.migration diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala index e723959c86..98d2aaf394 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala @@ -1,5 +1,5 @@ /* - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.reflogic diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmSpec.scala index 1d787c6572..a90d26ad8d 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmSpec.scala @@ -1,5 +1,5 @@ /* - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ // package akka.cluster.replication.transactionlog.writebehind.nosnapshot diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/snapshot/ReplicationTransactionLogWriteBehindSnapshotMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/snapshot/ReplicationTransactionLogWriteBehindSnapshotMultiJvmSpec.scala index 7f3a6fc683..fde113080e 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/snapshot/ReplicationTransactionLogWriteBehindSnapshotMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/snapshot/ReplicationTransactionLogWriteBehindSnapshotMultiJvmSpec.scala @@ -1,5 +1,5 @@ /* - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ // package akka.cluster.replication.transactionlog.writebehind.snapshot diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmSpec.scala index 2626d0fe8f..c2e6ed678b 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmSpec.scala @@ -1,5 +1,5 @@ /* - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ // package akka.cluster.replication.transactionlog.writethrough.nosnapshot diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/snapshot/ReplicationTransactionLogWriteThroughSnapshotMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/snapshot/ReplicationTransactionLogWriteThroughSnapshotMultiJvmSpec.scala index fe2231715e..3df29dd510 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/snapshot/ReplicationTransactionLogWriteThroughSnapshotMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/snapshot/ReplicationTransactionLogWriteThroughSnapshotMultiJvmSpec.scala @@ -1,5 +1,5 @@ /* - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ // package akka.cluster.replication.transactionlog.writethrough.snapshot diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_1/Random1ReplicaMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_1/Random1ReplicaMultiJvmSpec.scala index c9e0412be2..525a09467a 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_1/Random1ReplicaMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_1/Random1ReplicaMultiJvmSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.routing.random.replicationfactor_1 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmSpec.scala index cfe98d5680..c1a4175a09 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.routing.random.replicationfactor_3 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_1/RoundRobin1ReplicaMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_1/RoundRobin1ReplicaMultiJvmSpec.scala index 3605ba9d34..f8fd41b0cf 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_1/RoundRobin1ReplicaMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_1/RoundRobin1ReplicaMultiJvmSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.routing.roundrobin.replicationfactor_1 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_2/RoundRobin2ReplicasMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_2/RoundRobin2ReplicasMultiJvmSpec.scala index 1803ec2c83..b101a06f81 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_2/RoundRobin2ReplicasMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_2/RoundRobin2ReplicasMultiJvmSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.routing.roundrobin.replicationfactor_2 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmSpec.scala index ead1d693c0..f62b7d3e74 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmSpec.scala @@ -1,5 +1,5 @@ // /** -// * Copyright (C) 2009-2011 Typesafe Inc. +// * Copyright (C) 2009-2012 Typesafe Inc. // */ // package akka.cluster.routing.roundrobin.replicationfactor_3 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/sample/PingPongMultiJvmExample.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/sample/PingPongMultiJvmExample.scala index 9de63ee28a..c7e9aceaf1 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/sample/PingPongMultiJvmExample.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/sample/PingPongMultiJvmExample.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.sample diff --git a/akka-cluster/src/test/scala/akka/cluster/AsynchronousTransactionLogSpec.scala b/akka-cluster/src/test/scala/akka/cluster/AsynchronousTransactionLogSpec.scala index a43f6be62a..0d26befc4e 100644 --- a/akka-cluster/src/test/scala/akka/cluster/AsynchronousTransactionLogSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/AsynchronousTransactionLogSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster diff --git a/akka-cluster/src/test/scala/akka/cluster/SynchronousTransactionLogSpec.scala b/akka-cluster/src/test/scala/akka/cluster/SynchronousTransactionLogSpec.scala index 9bfb5a0257..3dc58d6c9a 100644 --- a/akka-cluster/src/test/scala/akka/cluster/SynchronousTransactionLogSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/SynchronousTransactionLogSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster diff --git a/akka-cluster/src/test/scala/akka/cluster/sample/ClusteredPingPongSample.scala b/akka-cluster/src/test/scala/akka/cluster/sample/ClusteredPingPongSample.scala index a0ba57c554..c242185450 100644 --- a/akka-cluster/src/test/scala/akka/cluster/sample/ClusteredPingPongSample.scala +++ b/akka-cluster/src/test/scala/akka/cluster/sample/ClusteredPingPongSample.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.sample diff --git a/akka-cluster/src/test/scala/akka/cluster/sample/ComputeGridSample.scala b/akka-cluster/src/test/scala/akka/cluster/sample/ComputeGridSample.scala index cb57df1718..daf817872e 100644 --- a/akka-cluster/src/test/scala/akka/cluster/sample/ComputeGridSample.scala +++ b/akka-cluster/src/test/scala/akka/cluster/sample/ComputeGridSample.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.sample diff --git a/akka-docs/general/code/akka/docs/config/ConfigDocSpec.scala b/akka-docs/general/code/akka/docs/config/ConfigDocSpec.scala index 3a94f2e4d1..3b7cb10ed2 100644 --- a/akka-docs/general/code/akka/docs/config/ConfigDocSpec.scala +++ b/akka-docs/general/code/akka/docs/config/ConfigDocSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.config diff --git a/akka-docs/java/code/akka/docs/actor/FaultHandlingTest.scala b/akka-docs/java/code/akka/docs/actor/FaultHandlingTest.scala index 8b3aee0759..03802d6968 100644 --- a/akka-docs/java/code/akka/docs/actor/FaultHandlingTest.scala +++ b/akka-docs/java/code/akka/docs/actor/FaultHandlingTest.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.actor import org.scalatest.junit.JUnitSuite diff --git a/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java b/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java index 132dc990ee..e402d44520 100644 --- a/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java +++ b/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.actor; diff --git a/akka-docs/java/code/akka/docs/actor/FirstUntypedActor.java b/akka-docs/java/code/akka/docs/actor/FirstUntypedActor.java index 509cb6c098..8fb12df4f7 100644 --- a/akka-docs/java/code/akka/docs/actor/FirstUntypedActor.java +++ b/akka-docs/java/code/akka/docs/actor/FirstUntypedActor.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.actor; diff --git a/akka-docs/java/code/akka/docs/actor/ImmutableMessage.java b/akka-docs/java/code/akka/docs/actor/ImmutableMessage.java index 1ff17b5544..41bc4eb0e5 100644 --- a/akka-docs/java/code/akka/docs/actor/ImmutableMessage.java +++ b/akka-docs/java/code/akka/docs/actor/ImmutableMessage.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.actor; diff --git a/akka-docs/java/code/akka/docs/actor/MyReceivedTimeoutUntypedActor.java b/akka-docs/java/code/akka/docs/actor/MyReceivedTimeoutUntypedActor.java index 91be082397..9131941851 100644 --- a/akka-docs/java/code/akka/docs/actor/MyReceivedTimeoutUntypedActor.java +++ b/akka-docs/java/code/akka/docs/actor/MyReceivedTimeoutUntypedActor.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.actor; diff --git a/akka-docs/java/code/akka/docs/actor/MyUntypedActor.java b/akka-docs/java/code/akka/docs/actor/MyUntypedActor.java index f16bc5d94b..93a817ef2c 100644 --- a/akka-docs/java/code/akka/docs/actor/MyUntypedActor.java +++ b/akka-docs/java/code/akka/docs/actor/MyUntypedActor.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.actor; diff --git a/akka-docs/java/code/akka/docs/actor/SchedulerDocTest.scala b/akka-docs/java/code/akka/docs/actor/SchedulerDocTest.scala index f46d54a15e..ecad03213e 100644 --- a/akka-docs/java/code/akka/docs/actor/SchedulerDocTest.scala +++ b/akka-docs/java/code/akka/docs/actor/SchedulerDocTest.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.actor diff --git a/akka-docs/java/code/akka/docs/actor/SchedulerDocTestBase.java b/akka-docs/java/code/akka/docs/actor/SchedulerDocTestBase.java index f9d1ad8799..7a58da0f5e 100644 --- a/akka-docs/java/code/akka/docs/actor/SchedulerDocTestBase.java +++ b/akka-docs/java/code/akka/docs/actor/SchedulerDocTestBase.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.actor; diff --git a/akka-docs/java/code/akka/docs/actor/TypedActorDocTest.scala b/akka-docs/java/code/akka/docs/actor/TypedActorDocTest.scala index bf92839761..476d570b4a 100644 --- a/akka-docs/java/code/akka/docs/actor/TypedActorDocTest.scala +++ b/akka-docs/java/code/akka/docs/actor/TypedActorDocTest.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.actor diff --git a/akka-docs/java/code/akka/docs/actor/TypedActorDocTestBase.java b/akka-docs/java/code/akka/docs/actor/TypedActorDocTestBase.java index 6726c3e6f0..72f950e2e7 100644 --- a/akka-docs/java/code/akka/docs/actor/TypedActorDocTestBase.java +++ b/akka-docs/java/code/akka/docs/actor/TypedActorDocTestBase.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.actor; diff --git a/akka-docs/java/code/akka/docs/actor/UntypedActorDocTest.scala b/akka-docs/java/code/akka/docs/actor/UntypedActorDocTest.scala index e0295ed209..e341914c8c 100644 --- a/akka-docs/java/code/akka/docs/actor/UntypedActorDocTest.scala +++ b/akka-docs/java/code/akka/docs/actor/UntypedActorDocTest.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.actor diff --git a/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java b/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java index a6cbc21338..b896f3691e 100644 --- a/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java +++ b/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.actor; diff --git a/akka-docs/java/code/akka/docs/actor/UntypedActorSwapper.java b/akka-docs/java/code/akka/docs/actor/UntypedActorSwapper.java index 7ecbab9ca3..b2fb98c305 100644 --- a/akka-docs/java/code/akka/docs/actor/UntypedActorSwapper.java +++ b/akka-docs/java/code/akka/docs/actor/UntypedActorSwapper.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.actor; diff --git a/akka-docs/java/code/akka/docs/agent/AgentDocJavaSpec.scala b/akka-docs/java/code/akka/docs/agent/AgentDocJavaSpec.scala index 3418830595..c3c0c296ed 100644 --- a/akka-docs/java/code/akka/docs/agent/AgentDocJavaSpec.scala +++ b/akka-docs/java/code/akka/docs/agent/AgentDocJavaSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.agent diff --git a/akka-docs/java/code/akka/docs/agent/AgentDocTest.java b/akka-docs/java/code/akka/docs/agent/AgentDocTest.java index fc966b8d5d..94ddef2c9f 100644 --- a/akka-docs/java/code/akka/docs/agent/AgentDocTest.java +++ b/akka-docs/java/code/akka/docs/agent/AgentDocTest.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.agent; diff --git a/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTest.scala b/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTest.scala index 0c28270394..8216c36757 100644 --- a/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTest.scala +++ b/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTest.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.dispatcher diff --git a/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTestBase.java b/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTestBase.java index 8b2006edd8..b1334bb9e5 100644 --- a/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTestBase.java +++ b/akka-docs/java/code/akka/docs/dispatcher/DispatcherDocTestBase.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.dispatcher; diff --git a/akka-docs/java/code/akka/docs/event/LoggingDocTest.scala b/akka-docs/java/code/akka/docs/event/LoggingDocTest.scala index 19d37e9914..ee44f502a4 100644 --- a/akka-docs/java/code/akka/docs/event/LoggingDocTest.scala +++ b/akka-docs/java/code/akka/docs/event/LoggingDocTest.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.event diff --git a/akka-docs/java/code/akka/docs/event/LoggingDocTestBase.java b/akka-docs/java/code/akka/docs/event/LoggingDocTestBase.java index de84ce2c50..1a1f7867c0 100644 --- a/akka-docs/java/code/akka/docs/event/LoggingDocTestBase.java +++ b/akka-docs/java/code/akka/docs/event/LoggingDocTestBase.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.event; diff --git a/akka-docs/java/code/akka/docs/extension/ExtensionDocTest.scala b/akka-docs/java/code/akka/docs/extension/ExtensionDocTest.scala index 558790002c..7b1b43b6ca 100644 --- a/akka-docs/java/code/akka/docs/extension/ExtensionDocTest.scala +++ b/akka-docs/java/code/akka/docs/extension/ExtensionDocTest.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.extension diff --git a/akka-docs/java/code/akka/docs/extension/ExtensionDocTestBase.java b/akka-docs/java/code/akka/docs/extension/ExtensionDocTestBase.java index 7b48cc7e91..dbc704a2b2 100644 --- a/akka-docs/java/code/akka/docs/extension/ExtensionDocTestBase.java +++ b/akka-docs/java/code/akka/docs/extension/ExtensionDocTestBase.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.extension; diff --git a/akka-docs/java/code/akka/docs/future/FutureDocTest.scala b/akka-docs/java/code/akka/docs/future/FutureDocTest.scala index 385886c553..8716beeced 100644 --- a/akka-docs/java/code/akka/docs/future/FutureDocTest.scala +++ b/akka-docs/java/code/akka/docs/future/FutureDocTest.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.future diff --git a/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java b/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java index 1e4e7fd350..3484c337e0 100644 --- a/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java +++ b/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.future; diff --git a/akka-docs/java/code/akka/docs/remoting/RemoteActorExample.java b/akka-docs/java/code/akka/docs/remoting/RemoteActorExample.java index 1a2d610aa7..f7686e744a 100644 --- a/akka-docs/java/code/akka/docs/remoting/RemoteActorExample.java +++ b/akka-docs/java/code/akka/docs/remoting/RemoteActorExample.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.remoting; diff --git a/akka-docs/java/code/akka/docs/serialization/SerializationDocTest.scala b/akka-docs/java/code/akka/docs/serialization/SerializationDocTest.scala index 538756d48a..26685dea80 100644 --- a/akka-docs/java/code/akka/docs/serialization/SerializationDocTest.scala +++ b/akka-docs/java/code/akka/docs/serialization/SerializationDocTest.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.serialization diff --git a/akka-docs/java/code/akka/docs/serialization/SerializationDocTestBase.java b/akka-docs/java/code/akka/docs/serialization/SerializationDocTestBase.java index 677fd051b5..1b917c8b5b 100644 --- a/akka-docs/java/code/akka/docs/serialization/SerializationDocTestBase.java +++ b/akka-docs/java/code/akka/docs/serialization/SerializationDocTestBase.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.serialization; diff --git a/akka-docs/java/code/akka/docs/transactor/CoordinatedCounter.java b/akka-docs/java/code/akka/docs/transactor/CoordinatedCounter.java index dca10b8984..4ee4ca4cfc 100644 --- a/akka-docs/java/code/akka/docs/transactor/CoordinatedCounter.java +++ b/akka-docs/java/code/akka/docs/transactor/CoordinatedCounter.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.transactor; diff --git a/akka-docs/java/code/akka/docs/transactor/Coordinator.java b/akka-docs/java/code/akka/docs/transactor/Coordinator.java index 6854ed99f6..8fa925824b 100644 --- a/akka-docs/java/code/akka/docs/transactor/Coordinator.java +++ b/akka-docs/java/code/akka/docs/transactor/Coordinator.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.transactor; diff --git a/akka-docs/java/code/akka/docs/transactor/Counter.java b/akka-docs/java/code/akka/docs/transactor/Counter.java index 0a6b7b2219..4e6f3d8a16 100644 --- a/akka-docs/java/code/akka/docs/transactor/Counter.java +++ b/akka-docs/java/code/akka/docs/transactor/Counter.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.transactor; diff --git a/akka-docs/java/code/akka/docs/transactor/FriendlyCounter.java b/akka-docs/java/code/akka/docs/transactor/FriendlyCounter.java index d70c653063..f643b5d892 100644 --- a/akka-docs/java/code/akka/docs/transactor/FriendlyCounter.java +++ b/akka-docs/java/code/akka/docs/transactor/FriendlyCounter.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.transactor; diff --git a/akka-docs/java/code/akka/docs/transactor/Increment.java b/akka-docs/java/code/akka/docs/transactor/Increment.java index ef1459a391..1d789c99e2 100644 --- a/akka-docs/java/code/akka/docs/transactor/Increment.java +++ b/akka-docs/java/code/akka/docs/transactor/Increment.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.transactor; diff --git a/akka-docs/java/code/akka/docs/transactor/Message.java b/akka-docs/java/code/akka/docs/transactor/Message.java index 4f182ba43a..6a8da72070 100644 --- a/akka-docs/java/code/akka/docs/transactor/Message.java +++ b/akka-docs/java/code/akka/docs/transactor/Message.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.transactor; diff --git a/akka-docs/java/code/akka/docs/transactor/TransactorDocJavaSpec.scala b/akka-docs/java/code/akka/docs/transactor/TransactorDocJavaSpec.scala index d8bf2bf692..4c61a156e8 100644 --- a/akka-docs/java/code/akka/docs/transactor/TransactorDocJavaSpec.scala +++ b/akka-docs/java/code/akka/docs/transactor/TransactorDocJavaSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.transactor diff --git a/akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java b/akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java index e6b45f675c..0b1662aaab 100644 --- a/akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java +++ b/akka-docs/java/code/akka/docs/transactor/TransactorDocTest.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.transactor; diff --git a/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocSpec.scala b/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocSpec.scala index 9e7a0fd6dc..827b4200d8 100644 --- a/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocSpec.scala +++ b/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.actor.mailbox diff --git a/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocTest.scala b/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocTest.scala index ec3d39cdb4..eba732e6a7 100644 --- a/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocTest.scala +++ b/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocTest.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.actor.mailbox diff --git a/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocTestBase.java b/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocTestBase.java index e2896c7bbc..834dc6f0fb 100644 --- a/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocTestBase.java +++ b/akka-docs/modules/code/akka/docs/actor/mailbox/DurableMailboxDocTestBase.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.actor.mailbox; diff --git a/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala b/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala index 8f0b344558..98072cabd1 100644 --- a/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.actor diff --git a/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala b/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala index fdb3f45178..6fb4d3c36b 100644 --- a/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.actor diff --git a/akka-docs/scala/code/akka/docs/actor/SchedulerDocSpec.scala b/akka-docs/scala/code/akka/docs/actor/SchedulerDocSpec.scala index f6daeab1fe..b6bffecb46 100644 --- a/akka-docs/scala/code/akka/docs/actor/SchedulerDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/actor/SchedulerDocSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.actor diff --git a/akka-docs/scala/code/akka/docs/actor/TypedActorDocSpec.scala b/akka-docs/scala/code/akka/docs/actor/TypedActorDocSpec.scala index 0d6ad1e648..f292b39a7c 100644 --- a/akka-docs/scala/code/akka/docs/actor/TypedActorDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/actor/TypedActorDocSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.actor diff --git a/akka-docs/scala/code/akka/docs/actor/UnnestedReceives.scala b/akka-docs/scala/code/akka/docs/actor/UnnestedReceives.scala index 482bc2bebf..194a958cce 100644 --- a/akka-docs/scala/code/akka/docs/actor/UnnestedReceives.scala +++ b/akka-docs/scala/code/akka/docs/actor/UnnestedReceives.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.actor diff --git a/akka-docs/scala/code/akka/docs/agent/AgentDocSpec.scala b/akka-docs/scala/code/akka/docs/agent/AgentDocSpec.scala index 95e4b04ea9..1e9ec1fd69 100644 --- a/akka-docs/scala/code/akka/docs/agent/AgentDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/agent/AgentDocSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.agent diff --git a/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala b/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala index ba46733cbb..2747da9f91 100644 --- a/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.dispatcher diff --git a/akka-docs/scala/code/akka/docs/event/LoggingDocSpec.scala b/akka-docs/scala/code/akka/docs/event/LoggingDocSpec.scala index 652c36af3f..cd6ecdb162 100644 --- a/akka-docs/scala/code/akka/docs/event/LoggingDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/event/LoggingDocSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.event diff --git a/akka-docs/scala/code/akka/docs/extension/ExtensionDocSpec.scala b/akka-docs/scala/code/akka/docs/extension/ExtensionDocSpec.scala index c5b0e7d517..e9ab9ac8e1 100644 --- a/akka-docs/scala/code/akka/docs/extension/ExtensionDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/extension/ExtensionDocSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.extension diff --git a/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala b/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala index 1b6a755ede..47a854d403 100644 --- a/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.future diff --git a/akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala b/akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala index 3a9f566ed8..98554a56f7 100644 --- a/akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala +++ b/akka-docs/scala/code/akka/docs/routing/RouterTypeExample.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.routing diff --git a/akka-docs/scala/code/akka/docs/routing/RouterViaConfigExample.scala b/akka-docs/scala/code/akka/docs/routing/RouterViaConfigExample.scala index 3f06e93b28..cc840eedc5 100644 --- a/akka-docs/scala/code/akka/docs/routing/RouterViaConfigExample.scala +++ b/akka-docs/scala/code/akka/docs/routing/RouterViaConfigExample.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.routing diff --git a/akka-docs/scala/code/akka/docs/routing/RouterViaProgramExample.scala b/akka-docs/scala/code/akka/docs/routing/RouterViaProgramExample.scala index 783a95d767..246ddbf5a2 100644 --- a/akka-docs/scala/code/akka/docs/routing/RouterViaProgramExample.scala +++ b/akka-docs/scala/code/akka/docs/routing/RouterViaProgramExample.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.routing diff --git a/akka-docs/scala/code/akka/docs/serialization/SerializationDocSpec.scala b/akka-docs/scala/code/akka/docs/serialization/SerializationDocSpec.scala index ea6bdf8c33..6baba425bc 100644 --- a/akka-docs/scala/code/akka/docs/serialization/SerializationDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/serialization/SerializationDocSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.serialization diff --git a/akka-docs/scala/code/akka/docs/testkit/PlainWordSpec.scala b/akka-docs/scala/code/akka/docs/testkit/PlainWordSpec.scala index 84a2719366..8df13da2ca 100644 --- a/akka-docs/scala/code/akka/docs/testkit/PlainWordSpec.scala +++ b/akka-docs/scala/code/akka/docs/testkit/PlainWordSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.testkit diff --git a/akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala b/akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala index 8eaba57c8c..c1815d36a0 100644 --- a/akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/testkit/TestkitDocSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.testkit diff --git a/akka-docs/scala/code/akka/docs/transactor/TransactorDocSpec.scala b/akka-docs/scala/code/akka/docs/transactor/TransactorDocSpec.scala index 246ea0a352..c45b252ffd 100644 --- a/akka-docs/scala/code/akka/docs/transactor/TransactorDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/transactor/TransactorDocSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.docs.transactor diff --git a/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala b/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala index 42bece7d6a..6715679eb5 100644 --- a/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor.mailbox diff --git a/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailboxExtension.scala b/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailboxExtension.scala index c26c774a7a..91e8085778 100644 --- a/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailboxExtension.scala +++ b/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailboxExtension.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor.mailbox diff --git a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FileBasedMailboxExtension.scala b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FileBasedMailboxExtension.scala index ded882e6e3..c09bfc9cb3 100644 --- a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FileBasedMailboxExtension.scala +++ b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FileBasedMailboxExtension.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor.mailbox diff --git a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FiledBasedMailbox.scala b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FiledBasedMailbox.scala index 88bfde0529..0f18f1ec58 100644 --- a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FiledBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FiledBasedMailbox.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor.mailbox diff --git a/akka-durable-mailboxes/akka-mailboxes-common/src/main/protocol/MailboxProtocol.proto b/akka-durable-mailboxes/akka-mailboxes-common/src/main/protocol/MailboxProtocol.proto index 914d40de67..96fab2bf95 100644 --- a/akka-durable-mailboxes/akka-mailboxes-common/src/main/protocol/MailboxProtocol.proto +++ b/akka-durable-mailboxes/akka-mailboxes-common/src/main/protocol/MailboxProtocol.proto @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ option java_package = "akka.actor.mailbox"; diff --git a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala index 6d6bf10eb0..7156a859c1 100644 --- a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala +++ b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor.mailbox diff --git a/akka-durable-mailboxes/akka-mailboxes-common/src/test/scala/akka/actor/mailbox/DurableMailboxSpec.scala b/akka-durable-mailboxes/akka-mailboxes-common/src/test/scala/akka/actor/mailbox/DurableMailboxSpec.scala index ce5e449f1e..444edfa72d 100644 --- a/akka-durable-mailboxes/akka-mailboxes-common/src/test/scala/akka/actor/mailbox/DurableMailboxSpec.scala +++ b/akka-durable-mailboxes/akka-mailboxes-common/src/test/scala/akka/actor/mailbox/DurableMailboxSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor.mailbox diff --git a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala index e6163d80a5..7da9dd7a3e 100644 --- a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala +++ b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor.mailbox diff --git a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala index f6507c41d2..4eb36eefe1 100644 --- a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor.mailbox diff --git a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailboxExtension.scala b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailboxExtension.scala index 14a5ce9b18..0176fc09f3 100644 --- a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailboxExtension.scala +++ b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailboxExtension.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor.mailbox diff --git a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoDurableMessage.scala b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoDurableMessage.scala index f14c97010b..a678031b26 100644 --- a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoDurableMessage.scala +++ b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoDurableMessage.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor.mailbox diff --git a/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailbox.scala b/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailbox.scala index 8c7587ec00..3c33663b73 100644 --- a/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailbox.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor.mailbox diff --git a/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailboxExtension.scala b/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailboxExtension.scala index 43445a0594..17ce479244 100644 --- a/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailboxExtension.scala +++ b/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailboxExtension.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor.mailbox diff --git a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/java/akka/cluster/zookeeper/ZooKeeperQueue.java b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/java/akka/cluster/zookeeper/ZooKeeperQueue.java index 4e06b64e6b..6a06234333 100644 --- a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/java/akka/cluster/zookeeper/ZooKeeperQueue.java +++ b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/java/akka/cluster/zookeeper/ZooKeeperQueue.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.zookeeper; diff --git a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala index 117acac383..4ac254cdc1 100644 --- a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor.mailbox diff --git a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailboxExtension.scala b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailboxExtension.scala index 8597c905b2..de2f2d586f 100644 --- a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailboxExtension.scala +++ b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailboxExtension.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor.mailbox diff --git a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/cluster/zookeeper/AkkaZkClient.scala b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/cluster/zookeeper/AkkaZkClient.scala index 4dbafdf6f1..f89af717e1 100644 --- a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/cluster/zookeeper/AkkaZkClient.scala +++ b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/cluster/zookeeper/AkkaZkClient.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.zookeeper diff --git a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/cluster/zookeeper/AkkaZooKeeper.scala b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/cluster/zookeeper/AkkaZooKeeper.scala index ce9e74c260..b5165ffb72 100644 --- a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/cluster/zookeeper/AkkaZooKeeper.scala +++ b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/cluster/zookeeper/AkkaZooKeeper.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.cluster.zookeeper diff --git a/akka-remote/src/main/protocol/RemoteProtocol.proto b/akka-remote/src/main/protocol/RemoteProtocol.proto index 557d4376a1..7fe287522d 100644 --- a/akka-remote/src/main/protocol/RemoteProtocol.proto +++ b/akka-remote/src/main/protocol/RemoteProtocol.proto @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ option java_package = "akka.remote"; diff --git a/akka-remote/src/main/scala/akka/package.scala b/akka-remote/src/main/scala/akka/package.scala index 4fae688d03..39f4c4f97e 100644 --- a/akka-remote/src/main/scala/akka/package.scala +++ b/akka-remote/src/main/scala/akka/package.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka diff --git a/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala b/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala index 23043c5303..0e653819ee 100644 --- a/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala +++ b/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote diff --git a/akka-remote/src/main/scala/akka/remote/Gossiper.scala b/akka-remote/src/main/scala/akka/remote/Gossiper.scala index a12e5ecab1..57742c746a 100644 --- a/akka-remote/src/main/scala/akka/remote/Gossiper.scala +++ b/akka-remote/src/main/scala/akka/remote/Gossiper.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote diff --git a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala index 56c8ec2ed8..878023c064 100644 --- a/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala +++ b/akka-remote/src/main/scala/akka/remote/MessageSerializer.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote diff --git a/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala b/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala index 6d3f340cb5..c108b54ae7 100644 --- a/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala +++ b/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote diff --git a/akka-remote/src/main/scala/akka/remote/Remote.scala b/akka-remote/src/main/scala/akka/remote/Remote.scala index 02d6d682b0..c5535833aa 100644 --- a/akka-remote/src/main/scala/akka/remote/Remote.scala +++ b/akka-remote/src/main/scala/akka/remote/Remote.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 397f110783..12aba95b85 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote diff --git a/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala b/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala index 404bf98c4c..deed34521a 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote diff --git a/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala b/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala index de3e0825ff..948e1c25d3 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteDeployer.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote diff --git a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala index eae0741844..ad4fd2625e 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote diff --git a/akka-remote/src/main/scala/akka/remote/VectorClock.scala b/akka-remote/src/main/scala/akka/remote/VectorClock.scala index c734bc0ea0..9da70111e9 100644 --- a/akka-remote/src/main/scala/akka/remote/VectorClock.scala +++ b/akka-remote/src/main/scala/akka/remote/VectorClock.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote diff --git a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala index ea240858d9..bfb30bc940 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote.netty diff --git a/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala b/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala index dffb874be6..83a52cbd7c 100644 --- a/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala +++ b/akka-remote/src/main/scala/akka/routing/RemoteRouterConfig.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.routing diff --git a/akka-remote/src/main/scala/akka/serialization/Compression.scala b/akka-remote/src/main/scala/akka/serialization/Compression.scala index df79fe1f22..0251ace964 100644 --- a/akka-remote/src/main/scala/akka/serialization/Compression.scala +++ b/akka-remote/src/main/scala/akka/serialization/Compression.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.serialization diff --git a/akka-remote/src/main/scala/akka/serialization/ProtobufSerializer.scala b/akka-remote/src/main/scala/akka/serialization/ProtobufSerializer.scala index af206a4234..f733e8188f 100644 --- a/akka-remote/src/main/scala/akka/serialization/ProtobufSerializer.scala +++ b/akka-remote/src/main/scala/akka/serialization/ProtobufSerializer.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.serialization diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/AkkaRemoteSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/AkkaRemoteSpec.scala index 2a38a33402..70afe668bb 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/AkkaRemoteSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/AkkaRemoteSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/Barrier.scala b/akka-remote/src/multi-jvm/scala/akka/remote/Barrier.scala index 6474179001..b11ec837b7 100755 --- a/akka-remote/src/multi-jvm/scala/akka/remote/Barrier.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/Barrier.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/FileBasedBarrier.scala b/akka-remote/src/multi-jvm/scala/akka/remote/FileBasedBarrier.scala index 68347715d2..a1773fc86e 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/FileBasedBarrier.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/FileBasedBarrier.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/MultiJvmSync.scala b/akka-remote/src/multi-jvm/scala/akka/remote/MultiJvmSync.scala index 2be4c156f2..c1e6080e6e 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/MultiJvmSync.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/MultiJvmSync.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/QuietReporter.scala b/akka-remote/src/multi-jvm/scala/akka/remote/QuietReporter.scala index 96ebb55f35..f323b75e23 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/QuietReporter.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/QuietReporter.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package org.scalatest.akka diff --git a/akka-remote/src/test/protocol/ProtobufProtocol.proto b/akka-remote/src/test/protocol/ProtobufProtocol.proto index 404e288e09..ccb92aa1e3 100644 --- a/akka-remote/src/test/protocol/ProtobufProtocol.proto +++ b/akka-remote/src/test/protocol/ProtobufProtocol.proto @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.actor; diff --git a/akka-remote/src/test/scala/akka/remote/NetworkFailureSpec.scala b/akka-remote/src/test/scala/akka/remote/NetworkFailureSpec.scala index e02b072e2e..3ed237d111 100644 --- a/akka-remote/src/test/scala/akka/remote/NetworkFailureSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/NetworkFailureSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote diff --git a/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala index 58199b4683..f65b4e5ed8 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteCommunicationSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote diff --git a/akka-remote/src/test/scala/akka/remote/RemoteDeployerSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteDeployerSpec.scala index 15016748d0..a11a635462 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteDeployerSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteDeployerSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote diff --git a/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala index 6418f93966..ebdea1e34b 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteRouterSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.remote diff --git a/akka-samples/akka-sample-hello-kernel/src/main/scala/sample/kernel/hello/HelloKernel.scala b/akka-samples/akka-sample-hello-kernel/src/main/scala/sample/kernel/hello/HelloKernel.scala index abf526484c..fd34795a95 100644 --- a/akka-samples/akka-sample-hello-kernel/src/main/scala/sample/kernel/hello/HelloKernel.scala +++ b/akka-samples/akka-sample-hello-kernel/src/main/scala/sample/kernel/hello/HelloKernel.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package sample.kernel.hello diff --git a/akka-samples/akka-sample-hello/src/main/scala/sample/hello/Main.scala b/akka-samples/akka-sample-hello/src/main/scala/sample/hello/Main.scala index 2921c2d27c..bbb5c36bc9 100644 --- a/akka-samples/akka-sample-hello/src/main/scala/sample/hello/Main.scala +++ b/akka-samples/akka-sample-hello/src/main/scala/sample/hello/Main.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package sample.hello diff --git a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/InternalMsg.java b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/InternalMsg.java index 37cca0dd53..b348aa9537 100644 --- a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/InternalMsg.java +++ b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/InternalMsg.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package sample.remote.calculator.java; diff --git a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JAdvancedCalculatorActor.java b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JAdvancedCalculatorActor.java index 2f4084e3f1..a8eaade104 100644 --- a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JAdvancedCalculatorActor.java +++ b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JAdvancedCalculatorActor.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package sample.remote.calculator.java; diff --git a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JCalcApp.java b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JCalcApp.java index 470c4d123a..1bd183fcf0 100644 --- a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JCalcApp.java +++ b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JCalcApp.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package sample.remote.calculator.java; diff --git a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JCalculatorApplication.java b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JCalculatorApplication.java index 9887e01511..8699e92ca2 100644 --- a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JCalculatorApplication.java +++ b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JCalculatorApplication.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package sample.remote.calculator.java; diff --git a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JCreationActor.java b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JCreationActor.java index 64200c26de..da1fb6df37 100644 --- a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JCreationActor.java +++ b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JCreationActor.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package sample.remote.calculator.java; diff --git a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JCreationApp.java b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JCreationApp.java index 67ac7f490a..40252694eb 100644 --- a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JCreationApp.java +++ b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JCreationApp.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package sample.remote.calculator.java; diff --git a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JCreationApplication.java b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JCreationApplication.java index ca2e961a5b..5b3d4f15a9 100644 --- a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JCreationApplication.java +++ b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JCreationApplication.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package sample.remote.calculator.java; diff --git a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JLookupActor.java b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JLookupActor.java index a690bc0024..568b8147ec 100644 --- a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JLookupActor.java +++ b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JLookupActor.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package sample.remote.calculator.java; diff --git a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JLookupApp.java b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JLookupApp.java index a854a1916d..542b0e2407 100644 --- a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JLookupApp.java +++ b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JLookupApp.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package sample.remote.calculator.java; diff --git a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JLookupApplication.java b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JLookupApplication.java index 6baf07a49a..5c2d050888 100644 --- a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JLookupApplication.java +++ b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JLookupApplication.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package sample.remote.calculator.java; diff --git a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JSimpleCalculatorActor.java b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JSimpleCalculatorActor.java index fdfe0ad646..1b3373d84c 100644 --- a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JSimpleCalculatorActor.java +++ b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/JSimpleCalculatorActor.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package sample.remote.calculator.java; diff --git a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/Op.java b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/Op.java index 1c461b0405..750d6dd705 100644 --- a/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/Op.java +++ b/akka-samples/akka-sample-remote/src/main/java/sample/remote/calculator/java/Op.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package sample.remote.calculator.java; diff --git a/akka-samples/akka-sample-remote/src/main/scala/sample/remote/calculator/CalculatorApplication.scala b/akka-samples/akka-sample-remote/src/main/scala/sample/remote/calculator/CalculatorApplication.scala index 89ec77e225..37868ed3a7 100644 --- a/akka-samples/akka-sample-remote/src/main/scala/sample/remote/calculator/CalculatorApplication.scala +++ b/akka-samples/akka-sample-remote/src/main/scala/sample/remote/calculator/CalculatorApplication.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package sample.remote.calculator diff --git a/akka-samples/akka-sample-remote/src/main/scala/sample/remote/calculator/CreationApplication.scala b/akka-samples/akka-sample-remote/src/main/scala/sample/remote/calculator/CreationApplication.scala index 3c4bfab638..a1a8cf4a03 100644 --- a/akka-samples/akka-sample-remote/src/main/scala/sample/remote/calculator/CreationApplication.scala +++ b/akka-samples/akka-sample-remote/src/main/scala/sample/remote/calculator/CreationApplication.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package sample.remote.calculator diff --git a/akka-samples/akka-sample-remote/src/main/scala/sample/remote/calculator/LookupApplication.scala b/akka-samples/akka-sample-remote/src/main/scala/sample/remote/calculator/LookupApplication.scala index d36a0134b7..f9cfb34688 100644 --- a/akka-samples/akka-sample-remote/src/main/scala/sample/remote/calculator/LookupApplication.scala +++ b/akka-samples/akka-sample-remote/src/main/scala/sample/remote/calculator/LookupApplication.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package sample.remote.calculator diff --git a/akka-samples/akka-sample-remote/src/main/scala/sample/remote/calculator/MathOp.scala b/akka-samples/akka-sample-remote/src/main/scala/sample/remote/calculator/MathOp.scala index f2a718cbdc..f223323f45 100644 --- a/akka-samples/akka-sample-remote/src/main/scala/sample/remote/calculator/MathOp.scala +++ b/akka-samples/akka-sample-remote/src/main/scala/sample/remote/calculator/MathOp.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package sample.remote.calculator diff --git a/akka-slf4j/src/main/scala/akka/event/slf4j/SLF4J.scala b/akka-slf4j/src/main/scala/akka/event/slf4j/SLF4J.scala index 91a6cd7bf2..72593d4f76 100644 --- a/akka-slf4j/src/main/scala/akka/event/slf4j/SLF4J.scala +++ b/akka-slf4j/src/main/scala/akka/event/slf4j/SLF4J.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.event.slf4j diff --git a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala index 191901b4ee..a3a1982e9f 100644 --- a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala +++ b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.testkit diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala index 85057790d2..35b5488ff5 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.testkit diff --git a/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala b/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala index e027620b61..f2457c437c 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestBarrier.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.testkit diff --git a/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala b/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala index 3bee246e11..d191b11eab 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.testkit diff --git a/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala index 7e53b5ea92..f486e3a5bb 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.testkit diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala index 85f36a567c..4155ea662d 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.testkit diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKitExtension.scala b/akka-testkit/src/main/scala/akka/testkit/TestKitExtension.scala index 8469b4b50c..89c40f48f4 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestKitExtension.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestKitExtension.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.testkit diff --git a/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala b/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala index 37e5e607fb..d57ed76877 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestLatch.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.testkit diff --git a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala index c7278eac55..971c794174 100644 --- a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.testkit diff --git a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala index 95082304db..cd7e2307b4 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.testkit diff --git a/akka-testkit/src/test/scala/akka/testkit/TestFSMRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestFSMRefSpec.scala index e7a7ec9e51..d2ec767504 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestFSMRefSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestFSMRefSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.testkit diff --git a/akka-transactor/src/main/scala/akka/transactor/Atomically.scala b/akka-transactor/src/main/scala/akka/transactor/Atomically.scala index 4995a6b8bd..b568fa18a0 100644 --- a/akka-transactor/src/main/scala/akka/transactor/Atomically.scala +++ b/akka-transactor/src/main/scala/akka/transactor/Atomically.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.transactor diff --git a/akka-transactor/src/main/scala/akka/transactor/Coordinated.scala b/akka-transactor/src/main/scala/akka/transactor/Coordinated.scala index f9ef8538be..bcc716e3c4 100644 --- a/akka-transactor/src/main/scala/akka/transactor/Coordinated.scala +++ b/akka-transactor/src/main/scala/akka/transactor/Coordinated.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.transactor diff --git a/akka-transactor/src/main/scala/akka/transactor/Transactor.scala b/akka-transactor/src/main/scala/akka/transactor/Transactor.scala index d33cd85e58..627a5ab249 100644 --- a/akka-transactor/src/main/scala/akka/transactor/Transactor.scala +++ b/akka-transactor/src/main/scala/akka/transactor/Transactor.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.transactor diff --git a/akka-transactor/src/main/scala/akka/transactor/UntypedTransactor.scala b/akka-transactor/src/main/scala/akka/transactor/UntypedTransactor.scala index 9a37f81915..679696f487 100644 --- a/akka-transactor/src/main/scala/akka/transactor/UntypedTransactor.scala +++ b/akka-transactor/src/main/scala/akka/transactor/UntypedTransactor.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.transactor diff --git a/akka-transactor/src/test/java/akka/transactor/ExpectedFailureException.java b/akka-transactor/src/test/java/akka/transactor/ExpectedFailureException.java index 5727317415..4094c08992 100644 --- a/akka-transactor/src/test/java/akka/transactor/ExpectedFailureException.java +++ b/akka-transactor/src/test/java/akka/transactor/ExpectedFailureException.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.transactor; diff --git a/akka-transactor/src/test/java/akka/transactor/Increment.java b/akka-transactor/src/test/java/akka/transactor/Increment.java index cdbd3fcfae..e8b33f9aeb 100644 --- a/akka-transactor/src/test/java/akka/transactor/Increment.java +++ b/akka-transactor/src/test/java/akka/transactor/Increment.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.transactor; diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedCounter.java b/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedCounter.java index 694a675d8e..5d59691b9e 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedCounter.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedCounter.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.transactor; diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java b/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java index 7fce881b2c..33c1e7653a 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedIncrementTest.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.transactor; diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedCounter.java b/akka-transactor/src/test/java/akka/transactor/UntypedCounter.java index f03f74b10f..bc3524845f 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedCounter.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedCounter.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.transactor; diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedFailer.java b/akka-transactor/src/test/java/akka/transactor/UntypedFailer.java index 1f9e6ff41c..1e9308b2c1 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedFailer.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedFailer.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.transactor; diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java b/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java index 9e2cf39f8d..d59b1315dc 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedTransactorTest.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.transactor; diff --git a/akka-transactor/src/test/scala/akka/transactor/CoordinatedIncrementSpec.scala b/akka-transactor/src/test/scala/akka/transactor/CoordinatedIncrementSpec.scala index c6e129f89b..dc993a2201 100644 --- a/akka-transactor/src/test/scala/akka/transactor/CoordinatedIncrementSpec.scala +++ b/akka-transactor/src/test/scala/akka/transactor/CoordinatedIncrementSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.transactor diff --git a/akka-transactor/src/test/scala/akka/transactor/FickleFriendsSpec.scala b/akka-transactor/src/test/scala/akka/transactor/FickleFriendsSpec.scala index 6f4e46de6a..9d39e50189 100644 --- a/akka-transactor/src/test/scala/akka/transactor/FickleFriendsSpec.scala +++ b/akka-transactor/src/test/scala/akka/transactor/FickleFriendsSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.transactor diff --git a/akka-transactor/src/test/scala/akka/transactor/JavaUntypedCoordinatedSpec.scala b/akka-transactor/src/test/scala/akka/transactor/JavaUntypedCoordinatedSpec.scala index 6c18959ce9..7179d09b03 100644 --- a/akka-transactor/src/test/scala/akka/transactor/JavaUntypedCoordinatedSpec.scala +++ b/akka-transactor/src/test/scala/akka/transactor/JavaUntypedCoordinatedSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.transactor diff --git a/akka-transactor/src/test/scala/akka/transactor/JavaUntypedTransactorSpec.scala b/akka-transactor/src/test/scala/akka/transactor/JavaUntypedTransactorSpec.scala index 7e3c0e8294..ea467cdf58 100644 --- a/akka-transactor/src/test/scala/akka/transactor/JavaUntypedTransactorSpec.scala +++ b/akka-transactor/src/test/scala/akka/transactor/JavaUntypedTransactorSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.transactor diff --git a/akka-transactor/src/test/scala/akka/transactor/TransactorSpec.scala b/akka-transactor/src/test/scala/akka/transactor/TransactorSpec.scala index db273cdac7..13b04b6bab 100644 --- a/akka-transactor/src/test/scala/akka/transactor/TransactorSpec.scala +++ b/akka-transactor/src/test/scala/akka/transactor/TransactorSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.transactor diff --git a/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java b/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java index 4be3bd3731..6d4f3053b2 100644 --- a/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java +++ b/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.tutorial.first.java; diff --git a/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala b/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala index 419761c4de..9b73b7620d 100644 --- a/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala +++ b/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.tutorial.first.scala diff --git a/akka-tutorials/akka-tutorial-first/src/test/scala/WorkerSpec.scala b/akka-tutorials/akka-tutorial-first/src/test/scala/WorkerSpec.scala index a9d2a202fd..86d0715d49 100644 --- a/akka-tutorials/akka-tutorial-first/src/test/scala/WorkerSpec.scala +++ b/akka-tutorials/akka-tutorial-first/src/test/scala/WorkerSpec.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.tutorial.first.scala diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 0ead5c3a8e..e03dda818e 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka diff --git a/scripts/generate_config_with_secure_cookie.sh b/scripts/generate_config_with_secure_cookie.sh index 6ef3d06e91..d36dfeac2a 100755 --- a/scripts/generate_config_with_secure_cookie.sh +++ b/scripts/generate_config_with_secure_cookie.sh @@ -3,7 +3,7 @@ exec scala "$0" "$@" !# /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ import java.security.{MessageDigest, SecureRandom} From d888ca5bff79bd97c3a1dfe9432b12658f32b39c Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 19 Jan 2012 19:08:51 +0100 Subject: [PATCH 050/152] Removing 'name' from dispatchers, just use id, much better --- .../src/test/scala/akka/actor/dispatch/ActorModelSpec.scala | 2 -- .../test/scala/akka/actor/dispatch/DispatchersSpec.scala | 6 ------ akka-actor/src/main/resources/reference.conf | 3 --- .../src/main/scala/akka/dispatch/AbstractDispatcher.scala | 5 ----- .../src/main/scala/akka/dispatch/BalancingDispatcher.scala | 3 +-- akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala | 5 ++--- akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala | 4 +--- .../src/main/scala/akka/dispatch/PinnedDispatcher.scala | 2 -- 8 files changed, 4 insertions(+), 26 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala index 71cf23ae7e..417a174f8b 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala @@ -450,7 +450,6 @@ object DispatcherModelSpec { private val instance: MessageDispatcher = { configureThreadPool(config, threadPoolConfig ⇒ new Dispatcher(prerequisites, - config.getString("name"), config.getString("id"), config.getInt("throughput"), Duration(config.getNanoseconds("throughput-deadline-time"), TimeUnit.NANOSECONDS), @@ -525,7 +524,6 @@ object BalancingDispatcherModelSpec { private val instance: MessageDispatcher = { configureThreadPool(config, threadPoolConfig ⇒ new BalancingDispatcher(prerequisites, - config.getString("name"), config.getString("id"), config.getInt("throughput"), Duration(config.getNanoseconds("throughput-deadline-time"), TimeUnit.NANOSECONDS), diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatchersSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatchersSpec.scala index d169981a5a..5d6dd65529 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatchersSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatchersSpec.scala @@ -58,11 +58,6 @@ class DispatchersSpec extends AkkaSpec(DispatchersSpec.config) { dispatcher.throughput must be(17) } - "use specific name" in { - val dispatcher = lookup("myapp.mydispatcher") - dispatcher.name must be("mydispatcher") - } - "use specific id" in { val dispatcher = lookup("myapp.mydispatcher") dispatcher.id must be("myapp.mydispatcher") @@ -95,7 +90,6 @@ class DispatchersSpec extends AkkaSpec(DispatchersSpec.config) { val d1 = lookup("myapp.mydispatcher") val d2 = lookup("myapp.mydispatcher") d1 must be === d2 - d1.name must be("mydispatcher") } } diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index 94efc34176..463c3121fe 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -155,9 +155,6 @@ akka { # parameters type = "Dispatcher" - # Name used in log messages and thread names. - name = "default-dispatcher" - # Toggles whether the threads created by this dispatcher should be daemons or not daemonic = off diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index 812cda9f9f..ff209c8c00 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -139,11 +139,6 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext */ protected[akka] def createMailbox(actor: ActorCell): Mailbox - /** - * Name of this dispatcher. - */ - def name: String - /** * Identifier of this dispatcher, corresponds to the full key * of the dispatcher configuration. diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index 92c5d1e967..cc2d7a5e96 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -31,14 +31,13 @@ import akka.util.Duration */ class BalancingDispatcher( _prerequisites: DispatcherPrerequisites, - _name: String, _id: String, throughput: Int, throughputDeadlineTime: Duration, mailboxType: MailboxType, config: ThreadPoolConfig, _shutdownTimeout: Duration) - extends Dispatcher(_prerequisites, _name, _id, throughput, throughputDeadlineTime, mailboxType, config, _shutdownTimeout) { + extends Dispatcher(_prerequisites, _id, throughput, throughputDeadlineTime, mailboxType, config, _shutdownTimeout) { val buddies = new ConcurrentSkipListSet[ActorCell](akka.util.Helpers.IdentityHashComparator) val rebalance = new AtomicBoolean(false) diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala index cc60decec9..0e72f01681 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala @@ -24,7 +24,6 @@ import java.util.concurrent._ */ class Dispatcher( _prerequisites: DispatcherPrerequisites, - val name: String, val id: String, val throughput: Int, val throughputDeadlineTime: Duration, @@ -33,7 +32,7 @@ class Dispatcher( val shutdownTimeout: Duration) extends MessageDispatcher(_prerequisites) { - protected[akka] val executorServiceFactory = executorServiceFactoryProvider.createExecutorServiceFactory(name) + protected[akka] val executorServiceFactory = executorServiceFactoryProvider.createExecutorServiceFactory(id) protected[akka] val executorService = new AtomicReference[ExecutorService](new ExecutorServiceDelegate { lazy val executor = executorServiceFactory.createExecutorService }) @@ -94,7 +93,7 @@ class Dispatcher( } else false } - override val toString = getClass.getSimpleName + "[" + name + "]" + override val toString = getClass.getSimpleName + "[" + id + "]" } object PriorityGenerator { diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala index 3dfffcdbbf..31258c540e 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala @@ -161,7 +161,6 @@ class DispatcherConfigurator(config: Config, prerequisites: DispatcherPrerequisi private val instance = configureThreadPool(config, threadPoolConfig ⇒ new Dispatcher(prerequisites, - config.getString("name"), config.getString("id"), config.getInt("throughput"), Duration(config.getNanoseconds("throughput-deadline-time"), TimeUnit.NANOSECONDS), @@ -186,7 +185,6 @@ class BalancingDispatcherConfigurator(config: Config, prerequisites: DispatcherP private val instance = configureThreadPool(config, threadPoolConfig ⇒ new BalancingDispatcher(prerequisites, - config.getString("name"), config.getString("id"), config.getInt("throughput"), Duration(config.getNanoseconds("throughput-deadline-time"), TimeUnit.NANOSECONDS), @@ -211,7 +209,7 @@ class PinnedDispatcherConfigurator(config: Config, prerequisites: DispatcherPrer */ override def dispatcher(): MessageDispatcher = configureThreadPool(config, threadPoolConfig ⇒ - new PinnedDispatcher(prerequisites, null, config.getString("name"), config.getString("id"), mailboxType, + new PinnedDispatcher(prerequisites, null, config.getString("id"), mailboxType, Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS), threadPoolConfig)).build diff --git a/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala index d89df72188..c88113fc64 100644 --- a/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala @@ -21,13 +21,11 @@ import java.util.concurrent.TimeUnit class PinnedDispatcher( _prerequisites: DispatcherPrerequisites, _actor: ActorCell, - _name: String, _id: String, _mailboxType: MailboxType, _shutdownTimeout: Duration, _threadPoolConfig: ThreadPoolConfig = ThreadPoolConfig()) extends Dispatcher(_prerequisites, - _name, _id, Int.MaxValue, Duration.Zero, From c4b430226696610024026c08b37cdf860bf00406 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 19 Jan 2012 19:48:57 +0100 Subject: [PATCH 051/152] Damn you ConfigSpec --- akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala index 191bbd101f..d633d942e5 100644 --- a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala @@ -24,7 +24,6 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference) { settings.ConfigVersion must equal("2.0-SNAPSHOT") getString("akka.actor.default-dispatcher.type") must equal("Dispatcher") - getString("akka.actor.default-dispatcher.name") must equal("default-dispatcher") getMilliseconds("akka.actor.default-dispatcher.keep-alive-time") must equal(60 * 1000) getDouble("akka.actor.default-dispatcher.core-pool-size-factor") must equal(3.0) getDouble("akka.actor.default-dispatcher.max-pool-size-factor") must equal(3.0) From 03bc15feb11a4bed8bf2efa42dde2c87cef9d0af Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 19 Jan 2012 20:55:27 +0100 Subject: [PATCH 052/152] #1692 - Adding config option for making the Scheduler daemonic --- .../src/test/scala/akka/config/ConfigSpec.scala | 9 +++++++++ akka-actor/src/main/resources/reference.conf | 1 + .../src/main/scala/akka/actor/ActorSystem.scala | 15 ++++++++------- .../src/main/scala/akka/actor/Scheduler.scala | 4 +++- 4 files changed, 21 insertions(+), 8 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala index d633d942e5..521e8d4d4a 100644 --- a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala @@ -36,9 +36,18 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference) { getMilliseconds("akka.actor.default-dispatcher.shutdown-timeout") must equal(1 * 1000) getInt("akka.actor.default-dispatcher.throughput") must equal(5) getMilliseconds("akka.actor.default-dispatcher.throughput-deadline-time") must equal(0) + getBoolean("akka.actor.serialize-messages") must equal(false) settings.SerializeAllMessages must equal(false) + getInt("akka.scheduler.ticksPerWheel") must equal(512) + settings.SchedulerTicksPerWheel must equal(512) + + getMilliseconds("akka.scheduler.tickDuration") must equal(100) + settings.SchedulerTickDuration must equal(100 millis) + + getBoolean("akka.scheduler.daemonic") must equal(true) + settings.SchedulerDaemonicity must equal(true) } } } diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index 463c3121fe..e90e4f41bf 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -271,5 +271,6 @@ akka { # For more information see: http://www.jboss.org/netty/ tickDuration = 100ms ticksPerWheel = 512 + daemonic = on } } diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 07ed860116..0a646709bc 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -96,6 +96,7 @@ object ActorSystem { final val SchedulerTickDuration = Duration(getMilliseconds("akka.scheduler.tickDuration"), MILLISECONDS) final val SchedulerTicksPerWheel = getInt("akka.scheduler.ticksPerWheel") + final val SchedulerDaemonicity = getBoolean("akka.scheduler.daemonic") if (ConfigVersion != Version) throw new ConfigurationException("Akka JAR version [" + Version + "] does not match the provided config version [" + ConfigVersion + "]") @@ -408,18 +409,18 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor * executed upon close(), the task may execute before its timeout. */ protected def createScheduler(): Scheduler = { - val threadFactory = new MonitorableThreadFactory("DefaultScheduler") - val hwt = new HashedWheelTimer(log, threadFactory, settings.SchedulerTickDuration, settings.SchedulerTicksPerWheel) + val hwt = new HashedWheelTimer(log, + new MonitorableThreadFactory("DefaultScheduler", settings.SchedulerDaemonicity), + settings.SchedulerTickDuration, + settings.SchedulerTicksPerWheel) // note that dispatcher is by-name parameter in DefaultScheduler constructor, // because dispatcher is not initialized when the scheduler is created - def safeDispatcher = { - if (dispatcher eq null) { + def safeDispatcher = dispatcher match { + case null ⇒ val exc = new IllegalStateException("Scheduler is using dispatcher before it has been initialized") log.error(exc, exc.getMessage) throw exc - } else { - dispatcher - } + case dispatcher ⇒ dispatcher } new DefaultScheduler(hwt, log, safeDispatcher) } diff --git a/akka-actor/src/main/scala/akka/actor/Scheduler.scala b/akka-actor/src/main/scala/akka/actor/Scheduler.scala index 4fc9bf681b..e89523b6bb 100644 --- a/akka-actor/src/main/scala/akka/actor/Scheduler.scala +++ b/akka-actor/src/main/scala/akka/actor/Scheduler.scala @@ -123,7 +123,9 @@ trait Cancellable { * if it does not enqueue a task. Once a task is queued, it MUST be executed or * returned from stop(). */ -class DefaultScheduler(hashedWheelTimer: HashedWheelTimer, log: LoggingAdapter, dispatcher: ⇒ MessageDispatcher) extends Scheduler with Closeable { +class DefaultScheduler(hashedWheelTimer: HashedWheelTimer, + log: LoggingAdapter, + dispatcher: ⇒ MessageDispatcher) extends Scheduler with Closeable { def schedule(initialDelay: Duration, delay: Duration, receiver: ActorRef, message: Any): Cancellable = { val continuousCancellable = new ContinuousCancellable From ced01769859510d0e32165101f84be6da9722a98 Mon Sep 17 00:00:00 2001 From: Ivan Porto Carrero Date: Thu, 19 Jan 2012 21:26:18 +0100 Subject: [PATCH 053/152] Adds rst docs and a package wide implicit to extend the actorsystem --- akka-docs/scala/zeromq.rst | 116 +++++++++++++++++++++++ akka-zeromq/src/main/scala/package.scala | 11 +++ 2 files changed, 127 insertions(+) create mode 100644 akka-docs/scala/zeromq.rst create mode 100644 akka-zeromq/src/main/scala/package.scala diff --git a/akka-docs/scala/zeromq.rst b/akka-docs/scala/zeromq.rst new file mode 100644 index 0000000000..b21c81d9c4 --- /dev/null +++ b/akka-docs/scala/zeromq.rst @@ -0,0 +1,116 @@ + +.. _zeromq-module: + +ZeroMQ +====== + +.. sidebar:: Contents + + .. contents:: :local: + +Module stability: **IN PROGRESS** + +Akka provides a ZeroMQ module which abstracts a ZeroMQ connection and therefore allows interaction between Akka actors to take place over ZeroMQ connections. The messages can be of a proprietary format or they can be defined using Protobuf. The socket actor is fault-tolerant by default and when you use the newSocket method to create new sockets it will properly reinitialize the socket. + +ZeroMQ is very opinionated when it comes to multi-threading so configuration option `akka.zeromq.socket-dispatcher` always needs to be configured to a PinnedDispatcher, because the actual ZeroMQ socket can only be accessed by the thread that created it. + +The ZeroMQ module for Akka is written against an API introduced in JZMQ, which uses JNI to interact with the native ZeroMQ library. Instead of using JZMQ, the module uses ZeroMQ binding for Scala that uses the native ZeroMQ library through JNA. In other words, the only native library that this module requires is the native ZeroMQ library. +The benefit of the scala library is that you don't need to compile and manage native dependencies at the cost of some runtime performance. The scala-bindings are compatible with the JNI bindings so they are a drop-in replacement, in case you really need to get that extra bit of performance out. + +Connection +---------- + +ZeroMQ supports multiple connectivity patterns, each aimed to meet a different set of requirements. Currently, this module supports publisher-subscriber connections and connections based on dealers and routers. For connecting or accepting connections, a socket must be created. Sockets are always created using ``akka.zeromq.ZeroMQ.newSocket``, for example: + +.. code-block:: scala + + import akka.zeromq._ + val socket = system.zeromq.newSocket(SocketType.Pub, Bind("tcp://127.0.0.1:1234")) + +will create a ZeroMQ Publisher socket that is Bound to the port 1234 on localhost. +Importing the akka.zeromq._ package ensures that the implicit zeromq method is available. +Similarly you can create a subscribtion socket, that subscribes to all messages from the publisher using: + +.. code-block:: scala + + val socket = system.zeromq.newSocket(SocketType.Sub, Connect("tcp://127.0.0.1:1234"), SubscribeAll) + +Also, a socket may be created with a listener that handles received messages as well as notifications: + +.. code-block:: scala + + val listener = system.actorOf(Props(new Actor { + def receive: Receive = { + case Connecting => ... + case _ => ... + } + })) + val socket = system.zeromq.newSocket(SocketType.Router, Listener(listener), Connect("tcp://localhost:1234")) + +The following sub-sections describe the supported connection patterns and how they can be used in an Akka environment. However, for a comprehensive discussion of connection patterns, please refer to `ZeroMQ -- The Guide `_. + +Publisher-subscriber connection +^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +In a publisher-subscriber (pub-sub) connection, the publisher accepts one or more subscribers. Each subscriber shall subscribe to one or more topics, whereas the publisher publishes messages to a set of topics. Also, a subscriber can subscribe to all available topics. + +When you're using zeromq pub/sub you should be aware that it needs multicast - check your cloud - to work properly and that the filtering of events for topics happens client side, so all events are always broadcasted to every subscriber. + +An actor is subscribed to a topic as follows: + +.. code-block:: scala + + val socket = system.zeromq.newSocket(SocketType.Sub, Listener(listener), Connect("tcp://localhost:1234"), Subscribe("the-topic")) + +Note that if the given string is empty (see below), the actor is subscribed to all topics. To unsubscribe from a topic you do the following: + +.. code-block:: scala + + socket ! Unsubscribe("SomeTopic1") + +In an Akka environment, pub-sub connections shall be used when an actor sends messages to one or more actors that do not interact with the actor that sent the message. The following piece of code creates a publisher actor, binds the socket, and sends a message to be published: + +.. code-block:: scala + + import akka.zeromq._ + val socket = system.zeromq.newSocket(SocketType.Pub, Bind("tcp://127.0.0.1:1234")) + socket ! Send("hello".getBytes) + +In the following code, the subscriber is configured to receive messages for all topics: + +.. code-block:: scala + + import akka.zeromq._ + val listener = system.actorOf(Props(new Actor { + def receive: Receive = { + case Connecting => ... + case _ => ... + } + })) + val socket = system.zeromq.newSocket(SocketType.Sub, Listener(listener), Connect("tcp://127.0.0.1:1234"), SubscribeAll) + +Router-Dealer connection +^^^^^^^^^^^^^^^^^^^^^^^^ + +While Pub/Sub is nice the real advantage of zeromq is that it is a "lego-box" for reliable messaging. And because there are so many integrations the multi-language support is fantastic. +When you're using ZeroMQ to integrate many systems you'll probably need to build your own ZeroMQ devices. This is where the router and dealer socket types come in handy. +With those socket types you can build your own reliable pub sub broker that uses TCP/IP and does publisher side filtering of events. + +To create a Router socket that has a high watermark configured, you would do: + +.. code-block:: scala + + import akka.zeromq._ + val listener = system.actorOf(Props(new Actor { + def receive: Receive = { + case Connecting => ... + case _ => ... + } + })) + val socket = system.zeromq.newSocket( + SocketType.Router, + Listener(listener), + Bind("tcp://127.0.0.1:1234"), + HWM(50000)) + +The akka-zeromq module accepts most if not all the available configuration options for a zeromq socket. \ No newline at end of file diff --git a/akka-zeromq/src/main/scala/package.scala b/akka-zeromq/src/main/scala/package.scala new file mode 100644 index 0000000000..e62a16738c --- /dev/null +++ b/akka-zeromq/src/main/scala/package.scala @@ -0,0 +1,11 @@ +package akka + +import actor.ActorSystem + +package object zeromq { + implicit def zeromqSystem(system: ActorSystem) = new { + def zeromq = system.extension(ZeroMQExtension) + } + + val SubscribeAll = Subscribe(Seq.empty) +} \ No newline at end of file From c3f41b3cd33eb0bf3a9e60cba03317d47c179989 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 19 Jan 2012 14:54:42 +0100 Subject: [PATCH 054/152] DOC: Added migration documentation for Scheduler. See #1406 --- .../project/migration-guide-1.3.x-2.0.x.rst | 43 ++++++++++++++++++- 1 file changed, 42 insertions(+), 1 deletion(-) diff --git a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst index 353e0c0ddb..4145fb0304 100644 --- a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst +++ b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst @@ -266,6 +266,48 @@ Documentation: * :ref:`event-bus-scala` * :ref:`event-bus-java` + +Scheduler +--------- + +The functionality of the scheduler is identical, but the API is slightly adjusted. + +v1.3:: + + //Schedules to send the "foo"-message to the testActor after 50ms + Scheduler.scheduleOnce(testActor, "foo", 50L, TimeUnit.MILLISECONDS) + + // Schedules periodic send of "foo"-message to the testActor after 1s inital delay, + // and then with 200ms between successive sends + Scheduler.schedule(testActor, "foo", 1000L, 200L, TimeUnit.MILLISECONDS) + + // Schedules a function to be executed (send the current time) to the testActor after 50ms + Scheduler.scheduleOnce({testActor ! System.currentTimeMillis}, 50L, TimeUnit.MILLISECONDS) + +v2.0:: + + //Schedules to send the "foo"-message to the testActor after 50ms + system.scheduler.scheduleOnce(50 milliseconds, testActor, "foo") + + // Schedules periodic send of "foo"-message to the testActor after 1s inital delay, + // and then with 200ms between successive sends + system.scheduler.schedule(1 second, 200 milliseconds, testActor, "foo") + + // Schedules a function to be executed (send the current time) to the testActor after 50ms + system.scheduler.scheduleOnce(50 milliseconds) { + testActor ! System.currentTimeMillis + } + + +The internal implementation of the scheduler is changed from +``java.util.concurrent.ScheduledExecutorService`` to a variant of +``org.jboss.netty.util.HashedWheelTimer``. + +Documentation: + + * :ref:`scheduler-scala` + * :ref:`scheduler-java` + Supervision ----------- @@ -379,6 +421,5 @@ More to be written * TypedActors * Routing * Remoting -* Scheduler * Configuration * ...? \ No newline at end of file From 5370371988e08317037be326e0ed3bff2ff1499d Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 19 Jan 2012 21:11:42 +0100 Subject: [PATCH 055/152] DOC: Added migration documentation for configuration. See #1406 --- .../project/migration-guide-1.3.x-2.0.x.rst | 54 ++++++++++++++++++- 1 file changed, 53 insertions(+), 1 deletion(-) diff --git a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst index 4145fb0304..07dea5974f 100644 --- a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst +++ b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst @@ -233,6 +233,59 @@ Documentation: * :ref:`untyped-actors-java` +Configuration +------------- + +A new, more powerful, configuration utility has been implemented. The format of the +configuration file is very similar to the format in v1.3. In addition it also supports +configuration files in json and properties format. The syntax is described in the +`HOCON `_ specification. + +v1.3:: + + include "other.conf" + + akka { + event-handler-level = "DEBUG" + } + +v2.0:: + + include "other" + + akka { + loglevel = "DEBUG" + } + +In v1.3 the default name of the configuration file was ``akka.conf``. +In v2.0 the default name is ``application.conf``. It is still loaded from classpath +or can be specified with java System properties (``-D`` command line arguments). + +v1.3:: + + -Dakka.config= + -Dakka.output.config.source=on + +v2.0:: + + -Dconfig.file= + -Dakka.logConfigOnStart=on + + +Several configuration properties have been changed, such as: + + * ``akka.event-handler-level`` => ``akka.loglevel`` + * dispatcher ``type`` values are changed + * ``akka.actor.throughput`` => ``akka.actor.default-dispatcher.throughput`` + * ``akka.remote.layer`` => ``akka.remote.transport`` + * the global ``time-unit`` property is removed, all durations are specified with duration unit in the property value, ``timeout = 5s`` + +Verify used configuration properties against the reference :ref:`configuration`. + +Documentation: + + * :ref:`configuration` + Logging ------- @@ -421,5 +474,4 @@ More to be written * TypedActors * Routing * Remoting -* Configuration * ...? \ No newline at end of file From eb3d0c57d75e19abfecb4eb4dd2b6cf7dad03873 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Thu, 19 Jan 2012 21:28:00 +0100 Subject: [PATCH 056/152] DOC: Added migration documentation for dispatchers. See #1406 --- .../project/migration-guide-1.3.x-2.0.x.rst | 42 ++++++++++++++++++- 1 file changed, 41 insertions(+), 1 deletion(-) diff --git a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst index 07dea5974f..219136ac95 100644 --- a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst +++ b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst @@ -437,6 +437,47 @@ Documentation: * :ref:`actors-scala` * :ref:`untyped-actors-java` +Dispatchers +----------- + +Dispatchers are defined in configuration instead of in code. + +v1.3:: + + // in code + val myDispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher(name) + .withNewThreadPoolWithLinkedBlockingQueueWithCapacity(100) + .setCorePoolSize(16) + .setMaxPoolSize(128) + .setKeepAliveTimeInMillis(60000) + .build + +v2.0:: + + // in config + my-dispatcher { + type = Dispatcher + core-pool-size-factor = 8.0 + max-pool-size-factor = 16.0 + mailbox-capacity = 100 + } + +The dispatcher is assigned to the actor in a different way. + +v1.3:: + + actorRef.dispatcher = MyGlobals.myDispatcher + slef.dispatcher = = MyGlobals.myDispatcher + +v2.0:: + + val myActor = system.actorOf(Props[MyActor].withDispatcher("my-dispatcher"), "myactor") + +Documentation: + + * :ref:`dispatchers-java` + * :ref:`dispatchers-scala` + Spawn ----- @@ -469,7 +510,6 @@ More to be written ------------------ * Futures -* Dispatchers * STM * TypedActors * Routing From 27da7c4d128243ca41025fad24bce5ab19e8e87c Mon Sep 17 00:00:00 2001 From: Peter Vlugter Date: Fri, 20 Jan 2012 11:31:28 +1300 Subject: [PATCH 057/152] Update java-friendly api for scala stm - move to japi.Stm - add newMap, newSet, newList methods with java conversions - add afterCompletion lifecycle callback --- .../docs/transactor/CoordinatedCounter.java | 6 +-- .../code/akka/docs/transactor/Counter.java | 6 +-- .../akka/docs/transactor/FriendlyCounter.java | 6 +-- .../stm/{JavaAPI.scala => japi/Stm.scala} | 51 ++++++++++++++++--- .../transactor/UntypedCoordinatedCounter.java | 9 ++-- .../java/akka/transactor/UntypedCounter.java | 9 ++-- .../scala/concurrent/stm/JavaAPITests.java | 35 ++++++------- 7 files changed, 75 insertions(+), 47 deletions(-) rename akka-transactor/src/main/scala/scala/concurrent/stm/{JavaAPI.scala => japi/Stm.scala} (66%) diff --git a/akka-docs/java/code/akka/docs/transactor/CoordinatedCounter.java b/akka-docs/java/code/akka/docs/transactor/CoordinatedCounter.java index f17e86ade0..a00d26ed88 100644 --- a/akka-docs/java/code/akka/docs/transactor/CoordinatedCounter.java +++ b/akka-docs/java/code/akka/docs/transactor/CoordinatedCounter.java @@ -8,10 +8,10 @@ package akka.docs.transactor; import akka.actor.*; import akka.transactor.*; import scala.concurrent.stm.Ref; -import static scala.concurrent.stm.JavaAPI.*; +import scala.concurrent.stm.japi.Stm; public class CoordinatedCounter extends UntypedActor { - private Ref.View count = newRef(0); + private Ref.View count = Stm.newRef(0); public void onReceive(Object incoming) throws Exception { if (incoming instanceof Coordinated) { @@ -24,7 +24,7 @@ public class CoordinatedCounter extends UntypedActor { } coordinated.atomic(new Runnable() { public void run() { - increment(count, 1); + Stm.increment(count, 1); } }); } diff --git a/akka-docs/java/code/akka/docs/transactor/Counter.java b/akka-docs/java/code/akka/docs/transactor/Counter.java index efe2aaed72..acd0d8f516 100644 --- a/akka-docs/java/code/akka/docs/transactor/Counter.java +++ b/akka-docs/java/code/akka/docs/transactor/Counter.java @@ -7,14 +7,14 @@ package akka.docs.transactor; //#class import akka.transactor.*; import scala.concurrent.stm.Ref; -import static scala.concurrent.stm.JavaAPI.*; +import scala.concurrent.stm.japi.Stm; public class Counter extends UntypedTransactor { - Ref.View count = newRef(0); + Ref.View count = Stm.newRef(0); public void atomically(Object message) { if (message instanceof Increment) { - increment(count, 1); + Stm.increment(count, 1); } } diff --git a/akka-docs/java/code/akka/docs/transactor/FriendlyCounter.java b/akka-docs/java/code/akka/docs/transactor/FriendlyCounter.java index 7ef31c5bea..fe3d759539 100644 --- a/akka-docs/java/code/akka/docs/transactor/FriendlyCounter.java +++ b/akka-docs/java/code/akka/docs/transactor/FriendlyCounter.java @@ -9,10 +9,10 @@ import akka.actor.*; import akka.transactor.*; import java.util.Set; import scala.concurrent.stm.Ref; -import static scala.concurrent.stm.JavaAPI.*; +import scala.concurrent.stm.japi.Stm; public class FriendlyCounter extends UntypedTransactor { - Ref.View count = newRef(0); + Ref.View count = Stm.newRef(0); @Override public Set coordinate(Object message) { if (message instanceof Increment) { @@ -25,7 +25,7 @@ public class FriendlyCounter extends UntypedTransactor { public void atomically(Object message) { if (message instanceof Increment) { - increment(count, 1); + Stm.increment(count, 1); } } diff --git a/akka-transactor/src/main/scala/scala/concurrent/stm/JavaAPI.scala b/akka-transactor/src/main/scala/scala/concurrent/stm/japi/Stm.scala similarity index 66% rename from akka-transactor/src/main/scala/scala/concurrent/stm/JavaAPI.scala rename to akka-transactor/src/main/scala/scala/concurrent/stm/japi/Stm.scala index 964664fe55..d9ed5a8330 100644 --- a/akka-transactor/src/main/scala/scala/concurrent/stm/JavaAPI.scala +++ b/akka-transactor/src/main/scala/scala/concurrent/stm/japi/Stm.scala @@ -1,14 +1,19 @@ /* scala-stm - (c) 2009-2011, Stanford University, PPL */ -package scala.concurrent.stm +package scala.concurrent.stm.japi import java.util.concurrent.Callable +import java.util.{ List ⇒ JList, Map ⇒ JMap, Set ⇒ JSet } +import scala.collection.JavaConversions +import scala.concurrent.stm +import scala.concurrent.stm._ import scala.runtime.AbstractFunction1 /** - * Java-friendly API. + * Java-friendly API for ScalaSTM. + * These methods can also be statically imported. */ -object JavaAPI { +object Stm { /** * Create a Ref with an initial value. Return a `Ref.View`, which does not @@ -20,38 +25,58 @@ object JavaAPI { /** * Create an empty TMap. Return a `TMap.View`, which does not require - * implicit transactions. + * implicit transactions. See newMap for included java conversion. * @return a new, empty `TMap.View` */ def newTMap[A, B](): TMap.View[A, B] = TMap.empty[A, B].single + /** + * Create an empty TMap. Return a `java.util.Map` view of this TMap. + * @return a new, empty `TMap.View` wrapped as a `java.util.Map`. + */ + def newMap[A, B](): JMap[A, B] = JavaConversions.mutableMapAsJavaMap(newTMap[A, B]) + /** * Create an empty TSet. Return a `TSet.View`, which does not require - * implicit transactions. + * implicit transactions. See newSet for included java conversion. * @return a new, empty `TSet.View` */ def newTSet[A](): TSet.View[A] = TSet.empty[A].single + /** + * Create an empty TSet. Return a `java.util.Set` view of this TSet. + * @return a new, empty `TSet.View` wrapped as a `java.util.Set`. + */ + def newSet[A](): JSet[A] = JavaConversions.mutableSetAsJavaSet(newTSet[A]) + /** * Create a TArray containing `length` elements. Return a `TArray.View`, - * which does not require implicit transactions. + * which does not require implicit transactions. See newList for included + * java conversion. * @param length the length of the `TArray.View` to be created * @return a new `TArray.View` containing `length` elements (initially null) */ def newTArray[A <: AnyRef](length: Int): TArray.View[A] = TArray.ofDim[A](length)(ClassManifest.classType(AnyRef.getClass)).single + /** + * Create an empty TArray. Return a `java.util.List` view of this Array. + * @param length the length of the `TArray.View` to be created + * @return a new, empty `TArray.View` wrapped as a `java.util.List`. + */ + def newList[A <: AnyRef](length: Int): JList[A] = JavaConversions.mutableSeqAsJavaList(newTArray[A](length)) + /** * Atomic block that takes a `Runnable`. * @param runnable the `Runnable` to run within a transaction */ - def atomic(runnable: Runnable): Unit = scala.concurrent.stm.atomic { txn ⇒ runnable.run } + def atomic(runnable: Runnable): Unit = stm.atomic { txn ⇒ runnable.run } /** * Atomic block that takes a `Callable`. * @param callable the `Callable` to run within a transaction * @return the value returned by the `Callable` */ - def atomic[A](callable: Callable[A]): A = scala.concurrent.stm.atomic { txn ⇒ callable.call } + def atomic[A](callable: Callable[A]): A = stm.atomic { txn ⇒ callable.call } /** * Transform the value stored by `ref` by applying the function `f`. @@ -109,4 +134,14 @@ object JavaAPI { val txn = Txn.findCurrent if (txn.isDefined) Txn.afterRollback(status ⇒ task.run)(txn.get) } + + /** + * Add a task to run after the current transaction has either rolled back + * or committed. + * @param task the `Runnable` task to run after transaction completion + */ + def afterCompletion(task: Runnable): Unit = { + val txn = Txn.findCurrent + if (txn.isDefined) Txn.afterCompletion(status ⇒ task.run)(txn.get) + } } diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedCounter.java b/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedCounter.java index 7c92930e02..435fb0df54 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedCounter.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedCoordinatedCounter.java @@ -7,15 +7,15 @@ package akka.transactor; import akka.actor.ActorRef; import akka.actor.Actors; import akka.actor.UntypedActor; -import static scala.concurrent.stm.JavaAPI.*; import scala.concurrent.stm.Ref; +import scala.concurrent.stm.japi.Stm; import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; public class UntypedCoordinatedCounter extends UntypedActor { private String name; - private Ref.View count = newRef(0); + private Ref.View count = Stm.newRef(0); public UntypedCoordinatedCounter(String name) { this.name = name; @@ -40,9 +40,8 @@ public class UntypedCoordinatedCounter extends UntypedActor { } coordinated.atomic(new Runnable() { public void run() { - increment(count, 1); - afterRollback(countDown); - afterCommit(countDown); + Stm.increment(count, 1); + Stm.afterCompletion(countDown); } }); } diff --git a/akka-transactor/src/test/java/akka/transactor/UntypedCounter.java b/akka-transactor/src/test/java/akka/transactor/UntypedCounter.java index 392bfbca42..e4e680f74b 100644 --- a/akka-transactor/src/test/java/akka/transactor/UntypedCounter.java +++ b/akka-transactor/src/test/java/akka/transactor/UntypedCounter.java @@ -7,8 +7,8 @@ package akka.transactor; import akka.actor.ActorRef; import akka.transactor.UntypedTransactor; import akka.transactor.SendTo; -import static scala.concurrent.stm.JavaAPI.*; import scala.concurrent.stm.Ref; +import scala.concurrent.stm.japi.Stm; import java.util.List; import java.util.Set; import java.util.concurrent.CountDownLatch; @@ -16,7 +16,7 @@ import java.util.concurrent.TimeUnit; public class UntypedCounter extends UntypedTransactor { private String name; - private Ref.View count = newRef(0); + private Ref.View count = Stm.newRef(0); public UntypedCounter(String name) { this.name = name; @@ -39,15 +39,14 @@ public class UntypedCounter extends UntypedTransactor { public void atomically(Object message) { if (message instanceof Increment) { - increment(count, 1); + Stm.increment(count, 1); final Increment increment = (Increment) message; Runnable countDown = new Runnable() { public void run() { increment.getLatch().countDown(); } }; - afterRollback(countDown); - afterCommit(countDown); + Stm.afterCompletion(countDown); } } diff --git a/akka-transactor/src/test/java/scala/concurrent/stm/JavaAPITests.java b/akka-transactor/src/test/java/scala/concurrent/stm/JavaAPITests.java index e2d0631590..63fb6abb74 100644 --- a/akka-transactor/src/test/java/scala/concurrent/stm/JavaAPITests.java +++ b/akka-transactor/src/test/java/scala/concurrent/stm/JavaAPITests.java @@ -5,13 +5,12 @@ package scala.concurrent.stm; import static org.junit.Assert.*; import org.junit.Test; -import scala.concurrent.stm.Ref; -import static scala.concurrent.stm.JavaAPI.*; +import scala.concurrent.stm.japi.Stm; +import static scala.concurrent.stm.japi.Stm.*; import scala.runtime.AbstractFunction1; import java.util.concurrent.Callable; -import static scala.collection.JavaConversions.*; import java.util.Map; import java.util.Set; import java.util.List; @@ -96,8 +95,7 @@ public class JavaAPITests { @Test public void createAndUseTMap() { - TMap.View tmap = newTMap(); - Map map = mutableMapAsJavaMap(tmap); + Map map = newMap(); map.put(1, "one"); map.put(2, "two"); assertEquals("one", map.get(1)); @@ -109,8 +107,7 @@ public class JavaAPITests { @Test(expected = TestException.class) public void failingTMapTransaction() { - TMap.View tmap = newTMap(); - final Map map = mutableMapAsJavaMap(tmap); + final Map map = newMap(); try { atomic(new Runnable() { public void run() { @@ -130,8 +127,7 @@ public class JavaAPITests { @Test public void createAndUseTSet() { - TSet.View tset = newTSet(); - Set set = mutableSetAsJavaSet(tset); + Set set = newSet(); set.add("one"); set.add("two"); assertTrue(set.contains("one")); @@ -146,16 +142,15 @@ public class JavaAPITests { @Test public void createAndUseTArray() { - TArray.View tarray = newTArray(3); - List seq = mutableSeqAsJavaList(tarray); - assertEquals(null, seq.get(0)); - assertEquals(null, seq.get(1)); - assertEquals(null, seq.get(2)); - seq.set(0, "zero"); - seq.set(1, "one"); - seq.set(2, "two"); - assertEquals("zero", seq.get(0)); - assertEquals("one", seq.get(1)); - assertEquals("two", seq.get(2)); + List list = newList(3); + assertEquals(null, list.get(0)); + assertEquals(null, list.get(1)); + assertEquals(null, list.get(2)); + list.set(0, "zero"); + list.set(1, "one"); + list.set(2, "two"); + assertEquals("zero", list.get(0)); + assertEquals("one", list.get(1)); + assertEquals("two", list.get(2)); } } From b2bfc8bec02ddc35c4b7718d7bb4388bcab83bcd Mon Sep 17 00:00:00 2001 From: Peter Vlugter Date: Fri, 20 Jan 2012 12:01:36 +1300 Subject: [PATCH 058/152] Update stm migration with changes to the java api --- .../project/migration-guide-1.3.x-2.0.x.rst | 35 ++++++++++++------- 1 file changed, 23 insertions(+), 12 deletions(-) diff --git a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst index 2749f0107c..e00c116cd0 100644 --- a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst +++ b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst @@ -403,11 +403,17 @@ To use ScalaSTM the import from Scala is:: Java ~~~~ -For Java there is a special JavaAPI helper object that can be statically -imported, along with any other imports that might be needed:: +For Java there is a special helper object with Java-friendly methods:: + + import scala.concurrent.stm.japi.Stm; + +These methods can also be statically imported:: + + import static scala.concurrent.stm.japi.Stm.*; + +Other imports that are needed are in the stm package, particularly ``Ref``:: import scala.concurrent.stm.Ref; - import static scala.concurrent.stm.JavaAPI.*; Transactions ^^^^^^^^^^^^ @@ -440,7 +446,7 @@ for more information. Java ~~~~ -In the ScalaSTM JavaAPI helpers there are atomic methods which accept +In the ScalaSTM Java API helpers there are atomic methods which accept ``java.lang.Runnable`` and ``java.util.concurrent.Callable``. v1.3:: @@ -461,7 +467,7 @@ v1.3:: v2.0:: - import static scala.concurrent.stm.JavaAPI.*; + import static scala.concurrent.stm.japi.Stm.atomic; import java.util.concurrent.Callable; atomic(new Runnable() { @@ -546,7 +552,7 @@ Java As ``Ref.View`` in ScalaSTM does not require implicit transactions, this is more easily used from Java. ``Ref`` could be used, but requires explicit threading of -transactions. There are helper methods in ``JavaAPI`` for creating ``Ref.View`` +transactions. There are helper methods in ``japi.Stm`` for creating ``Ref.View`` references. v1.3:: @@ -555,7 +561,7 @@ v1.3:: v2.0:: - Ref.View ref = newRef(0); + Ref.View ref = Stm.newRef(0); The ``set`` and ``get`` methods work the same way for both versions. @@ -570,7 +576,7 @@ v2.0:: ref.set(1); // set new value There are also ``transform``, ``getAndTransform``, and ``transformAndGet`` -methods in ``JavaAPI`` which accept ``scala.runtime.AbstractFunction1``. +methods in ``japi.Stm`` which accept ``scala.runtime.AbstractFunction1``. There are ``increment`` helper methods for ``Ref.View`` and ``Ref.View`` references. @@ -611,16 +617,21 @@ Java Rather than using the ``deferred`` and ``compensating`` methods in ``akka.stm.StmUtils``, use the ``afterCommit`` and ``afterRollback`` methods in -``scala.concurrent.stm.JavaAPI``, which behave in the same way and accept +``scala.concurrent.stm.japi.Stm``, which behave in the same way and accept ``Runnable``. Transactional Datastructures ^^^^^^^^^^^^^^^^^^^^^^^^^^^^ In ScalaSTM see ``TMap``, ``TSet``, and ``TArray`` for transactional -datastructures. There are helper methods for creating these in -``JavaAPI``. These datastructure implement the ``scala.collection`` interfaces -and can also be used from Java with Scala's ``JavaConversions``. +datastructures. + +There are helper methods for creating these from Java in ``japi.Stm``: +``newTMap``, ``newTSet``, and ``newTArray``. These datastructures implement the +``scala.collection`` interfaces and can also be used from Java with Scala's +``JavaConversions``. There are helper methods that apply the conversions, +returning ``java.util`` ``Map``, ``Set``, and ``List``: ``newMap``, ``newSet``, +and ``newList``. More to be written From adc29406fd90311059411acba75e25cda7829fe7 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Thu, 19 Jan 2012 16:11:28 -0700 Subject: [PATCH 059/152] Throw proper Exceptions instead of sys.error() --- akka-actor/src/main/scala/akka/actor/IO.scala | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/IO.scala b/akka-actor/src/main/scala/akka/actor/IO.scala index 1911770a63..50574ef662 100644 --- a/akka-actor/src/main/scala/akka/actor/IO.scala +++ b/akka-actor/src/main/scala/akka/actor/IO.scala @@ -25,9 +25,14 @@ import com.eaio.uuid.UUID /** * IO messages and iteratees. + * + * This is still in an experimental state and is subject to change until it + * has received more real world testing. */ object IO { + final class DivergentIterateeException extends Exception("Iteratees should not return a continuation when receiving EOF") + /** * An immutable handle to a Java NIO Channel. Contains a reference to the * [[akka.actor.ActorRef]] that will receive events related to the Channel, @@ -41,10 +46,10 @@ object IO { def uuid: UUID override lazy val hashCode = scala.runtime.ScalaRunTime._hashCode(this) - def asReadable: ReadHandle = sys error "Not readable" - def asWritable: WriteHandle = sys error "Not writable" - def asSocket: SocketHandle = sys error "Not a socket" - def asServer: ServerHandle = sys error "Not a server" + def asReadable: ReadHandle = throw new ClassCastException(this.toString + " is not a ReadHandle") + def asWritable: WriteHandle = throw new ClassCastException(this.toString + " is not a WriteHandle") + def asSocket: SocketHandle = throw new ClassCastException(this.toString + " is not a SocketHandle") + def asServer: ServerHandle = throw new ClassCastException(this.toString + " is not a ServerHandle") /** * Sends a request to the [[akka.actor.IOManager]] to close the Channel @@ -271,7 +276,7 @@ object IO { */ final def get: A = this(EOF(None))._1 match { case Done(value) ⇒ value - case Cont(_, None) ⇒ sys.error("Divergent Iteratee") + case Cont(_, None) ⇒ throw new DivergentIterateeException case Cont(_, Some(err)) ⇒ throw err } @@ -802,6 +807,7 @@ final class IOManagerActor extends Actor { } } } 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)) From 6519b3f246e9c60ad205ef58be4bae41b6be8a88 Mon Sep 17 00:00:00 2001 From: Ivan Porto Carrero Date: Fri, 20 Jan 2012 00:34:36 +0100 Subject: [PATCH 060/152] Adds scaladocs and removes some options that made no sense in the current implementation --- .../akka/zeromq/ConcurrentSocketActor.scala | 10 +- .../src/main/scala/akka/zeromq/Requests.scala | 137 ----- .../main/scala/akka/zeromq/SocketOption.scala | 509 ++++++++++++++++++ ...zer.scala => ZMQMessageDeserializer.scala} | 7 + .../scala/akka/zeromq/ZeroMQExtension.scala | 39 +- akka-zeromq/src/main/scala/package.scala | 21 +- 6 files changed, 574 insertions(+), 149 deletions(-) delete mode 100644 akka-zeromq/src/main/scala/akka/zeromq/Requests.scala create mode 100644 akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala rename akka-zeromq/src/main/scala/akka/zeromq/{Deserializer.scala => ZMQMessageDeserializer.scala} (67%) diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala index 8585883868..3f6caad411 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala @@ -24,13 +24,16 @@ private[zeromq] object ConcurrentSocketActor { private case object ClearPoll private case class PollError(ex: Throwable) + private class NoSocketHandleException() extends Exception("Couldn't create a zeromq socket.") + + private val DefaultContext = Context() } private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends Actor { import ConcurrentSocketActor._ private val noBytes = Array[Byte]() private val zmqContext = { - params collectFirst { case c: Context ⇒ c } getOrElse Context() + params collectFirst { case c: Context ⇒ c } getOrElse DefaultContext } private val deserializer = deserializerFromParams @@ -83,8 +86,6 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A case RecoveryInterval(value) ⇒ socket.setRecoveryInterval(value) case MulticastLoop(value) ⇒ socket.setMulticastLoop(value) case MulticastHops(value) ⇒ socket.setMulticastHops(value) - case ReceiveTimeOut(value) ⇒ socket.setReceiveTimeOut(value) - case SendTimeOut(value) ⇒ socket.setSendTimeOut(value) case SendBufferSize(value) ⇒ socket.setSendBufferSize(value) case ReceiveBufferSize(value) ⇒ socket.setReceiveBufferSize(value) case Linger ⇒ sender ! socket.getLinger @@ -101,11 +102,8 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A case RecoveryInterval ⇒ sender ! socket.getRecoveryInterval case MulticastLoop ⇒ sender ! socket.hasMulticastLoop case MulticastHops ⇒ sender ! socket.getMulticastHops - case ReceiveTimeOut ⇒ sender ! socket.getReceiveTimeOut - case SendTimeOut ⇒ sender ! socket.getSendTimeOut case SendBufferSize ⇒ sender ! socket.getSendBufferSize case ReceiveBufferSize ⇒ sender ! socket.getReceiveBufferSize - case ReceiveMore ⇒ sender ! socket.hasReceiveMore case FileDescriptor ⇒ sender ! socket.getFD } diff --git a/akka-zeromq/src/main/scala/akka/zeromq/Requests.scala b/akka-zeromq/src/main/scala/akka/zeromq/Requests.scala deleted file mode 100644 index 371d2a3dde..0000000000 --- a/akka-zeromq/src/main/scala/akka/zeromq/Requests.scala +++ /dev/null @@ -1,137 +0,0 @@ -/** - * Copyright (C) 2009-2011 Typesafe Inc. - */ -package akka.zeromq - -import com.google.protobuf.Message -import org.zeromq.{ ZMQ ⇒ JZMQ } -import akka.actor.ActorRef -import akka.util.duration._ -import akka.util.Duration - -sealed trait Request -sealed trait SocketOption extends Request -sealed trait SocketMeta extends SocketOption -sealed trait SocketConnectOption extends SocketOption { - def endpoint: String -} -sealed trait PubSubOption extends SocketOption { - def payload: Seq[Byte] -} -sealed trait SocketOptionQuery extends Request - -case class Connect(endpoint: String) extends SocketConnectOption - -object Context { - def apply(numIoThreads: Int = 1) = new Context(numIoThreads) -} -class Context(numIoThreads: Int) extends SocketMeta { - private val context = JZMQ.context(numIoThreads) - def socket(socketType: SocketType.ZMQSocketType) = { - context.socket(socketType.id) - } - def poller = { - context.poller - } - def term = { - context.term - } -} - -trait Deserializer extends SocketOption { - def apply(frames: Seq[Frame]): Any -} - -object SocketType { - abstract class ZMQSocketType(val id: Int) extends SocketMeta - object Pub extends ZMQSocketType(JZMQ.PUB) - object Sub extends ZMQSocketType(JZMQ.SUB) - object Dealer extends ZMQSocketType(JZMQ.DEALER) - object Router extends ZMQSocketType(JZMQ.ROUTER) -} - -case class Listener(listener: ActorRef) extends SocketMeta -case class PollDispatcher(name: String) extends SocketMeta -case class PollTimeoutDuration(duration: Duration = 100 millis) extends SocketMeta - -case class Bind(endpoint: String) extends SocketConnectOption -private[zeromq] case object Close extends Request - -case class Subscribe(payload: Seq[Byte]) extends PubSubOption -object Subscribe { - def apply(topic: String): Subscribe = new Subscribe(topic.getBytes) -} - -case class Unsubscribe(payload: Seq[Byte]) extends PubSubOption -object Unsubscribe { - def apply(topic: String): Unsubscribe = Unsubscribe(topic.getBytes) -} - -case class Send(frames: Seq[Frame]) extends Request - -case class ZMQMessage(frames: Seq[Frame]) { - def firstFrameAsString = new String(frames.head.payload.toArray) -} -object ZMQMessage { - def apply(bytes: Array[Byte]): ZMQMessage = ZMQMessage(Seq(Frame(bytes))) - def apply(message: Message): ZMQMessage = ZMQMessage(message.toByteArray) -} - -case class Linger(value: Long) extends SocketOption -object Linger extends SocketOptionQuery - -case class ReconnectIVL(value: Long) extends SocketOption -object ReconnectIVL extends SocketOptionQuery - -case class Backlog(value: Long) extends SocketOption -object Backlog extends SocketOptionQuery - -case class ReconnectIVLMax(value: Long) extends SocketOption -object ReconnectIVLMax extends SocketOptionQuery - -case class MaxMsgSize(value: Long) extends SocketOption -object MaxMsgSize extends SocketOptionQuery - -case class SndHWM(value: Long) extends SocketOption -object SndHWM extends SocketOptionQuery - -case class RcvHWM(value: Long) extends SocketOption -object RcvHWM extends SocketOptionQuery - -case class HWM(value: Long) extends SocketOption - -case class Swap(value: Long) extends SocketOption -object Swap extends SocketOptionQuery - -case class Affinity(value: Long) extends SocketOption -object Affinity extends SocketOptionQuery - -case class Identity(value: Array[Byte]) extends SocketOption -object Identity extends SocketOptionQuery - -case class Rate(value: Long) extends SocketOption -object Rate extends SocketOptionQuery - -case class RecoveryInterval(value: Long) extends SocketOption -object RecoveryInterval extends SocketOptionQuery - -case class MulticastLoop(value: Boolean) extends SocketOption -object MulticastLoop extends SocketOptionQuery - -case class MulticastHops(value: Long) extends SocketOption -object MulticastHops extends SocketOptionQuery - -case class ReceiveTimeOut(value: Long) extends SocketOption -object ReceiveTimeOut extends SocketOptionQuery - -case class SendTimeOut(value: Long) extends SocketOption -object SendTimeOut extends SocketOptionQuery - -case class SendBufferSize(value: Long) extends SocketOption -object SendBufferSize extends SocketOptionQuery - -case class ReceiveBufferSize(value: Long) extends SocketOption -object ReceiveBufferSize extends SocketOptionQuery - -object ReceiveMore extends SocketOptionQuery -object FileDescriptor extends SocketOptionQuery diff --git a/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala b/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala new file mode 100644 index 0000000000..457129ea5c --- /dev/null +++ b/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala @@ -0,0 +1,509 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka.zeromq + +import com.google.protobuf.Message +import org.zeromq.{ ZMQ ⇒ JZMQ } +import akka.actor.ActorRef +import akka.util.duration._ +import akka.util.Duration + +/** + * Marker trait representing request messages for zeromq + */ +sealed trait Request + +/** + * Marker trait representing the base for all socket options + */ +sealed trait SocketOption extends Request + +/** + * Marker trait representing the base for all meta operations for a socket + * such as the context, listener, socket type and poller dispatcher + */ +sealed trait SocketMeta extends SocketOption + +/** + * A base trait for connection options for a ZeroMQ socket + */ +sealed trait SocketConnectOption extends SocketOption { + def endpoint: String +} + +/** + * A base trait for pubsub options for the ZeroMQ socket + */ +sealed trait PubSubOption extends SocketOption { + def payload: Seq[Byte] +} + +/** + * A marker trait to group option queries together + */ +sealed trait SocketOptionQuery extends Request + +/** + * This socket should be a client socket and connect to the specified endpoint + * + * @param endpoint an uri like tcp://127.0.0.1.5432 + */ +case class Connect(endpoint: String) extends SocketConnectOption + +/** + * Companion object for a ZeroMQ I/O thread pool + */ +object Context { + def apply(numIoThreads: Int = 1) = new Context(numIoThreads) +} + +/** + * Represents an I/O thread pool for ZeroMQ sockets. + * By default the ZeroMQ module uses a thread pool with 1 thread. For most applications that should be + * sufficient + * + * @param numIoThreads + */ +class Context(numIoThreads: Int) extends SocketMeta { + private val context = JZMQ.context(numIoThreads) + def socket(socketType: SocketType.ZMQSocketType) = { + context.socket(socketType.id) + } + def poller = { + context.poller + } + def term = { + context.term + } +} + +/** + * A base trait for message deserializers + */ +trait Deserializer extends SocketOption { + def apply(frames: Seq[Frame]): Any +} + +/** + * The different socket types you can create with zeromq + */ +object SocketType { + + abstract class ZMQSocketType(val id: Int) extends SocketMeta + + /** + * A Publisher socket + */ + object Pub extends ZMQSocketType(JZMQ.PUB) + + /** + * A subscriber socket + */ + object Sub extends ZMQSocketType(JZMQ.SUB) + + /** + * A dealer socket + */ + object Dealer extends ZMQSocketType(JZMQ.DEALER) + + /** + * A router socket + */ + object Router extends ZMQSocketType(JZMQ.ROUTER) + + /** + * A request socket + */ + object Req extends ZMQSocketType(JZMQ.REQ) + + /** + * A reply socket + */ + object Rep extends ZMQSocketType(JZMQ.REP) + + /** + * A push socket + */ + object Push extends ZMQSocketType(JZMQ.PUSH) + + /** + * A pull socket + */ + object Pull extends ZMQSocketType(JZMQ.PULL) + + /** + * A Pair socket + */ + object Pair extends ZMQSocketType(JZMQ.PAIR) +} + +/** + * An option containing the listener for the socket + * @param listener + */ +case class Listener(listener: ActorRef) extends SocketMeta + +/** + * An option containing the configuration key for the poller loop dispatcher + * @param name + */ +case class PollDispatcher(name: String) extends SocketMeta + +/** + * An option containing the duration a poll cycle should wait for a message before it loops + * @param duration + */ +case class PollTimeoutDuration(duration: Duration = 100 millis) extends SocketMeta + +/** + * Start listening with this server socket on the specified address + * + * @param endpoint + */ +case class Bind(endpoint: String) extends SocketConnectOption +private[zeromq] case object Close extends Request + +/** + * The [[akka.zeromq.Subscribe]] option shall establish a new message filter on a [[akka.zeromq.SocketType.Pub]] socket. + * Newly created [[akka.zeromq.SocketType.Sub]] sockets shall filter out all incoming messages, + * therefore you should send this option to establish an initial message filter. + * + * An empty payload of length zero shall subscribe to all incoming messages. + * A non-empty payload shall subscribe to all messages beginning with the specified prefix. + * Multiple filters may be attached to a single [[akka.zeromq.SocketType.Sub]] socket, + * in which case a message shall be accepted if it matches at least one filter. + * + * @param payload the topic to subscribe to + */ +case class Subscribe(payload: Seq[Byte]) extends PubSubOption +object Subscribe { + def apply(topic: String): Subscribe = new Subscribe(topic.getBytes) +} + +/** + * The [[akka.zeromq.Unsubscribe]] option shall remove an existing message filter + * on a [[akka.zeromq.SocketType.Sub]] socket. The filter specified must match an existing filter + * previously established with the [[akka.zeromq.Subscribe]] option. If the socket has several instances of the + * same filter attached the [[akka.zeromq.Unsubscribe]] option shall remove only one instance, leaving the rest + * in place and functional. + * + * @param payload + */ +case class Unsubscribe(payload: Seq[Byte]) extends PubSubOption +object Unsubscribe { + def apply(topic: String): Unsubscribe = Unsubscribe(topic.getBytes) +} + +/** + * Send a message over the zeromq socket + * @param frames + */ +case class Send(frames: Seq[Frame]) extends Request + +/** + * A message received over the zeromq socket + * @param frames + */ +case class ZMQMessage(frames: Seq[Frame]) { + def firstFrameAsString = new String(frames.head.payload.toArray) +} +object ZMQMessage { + def apply(bytes: Array[Byte]): ZMQMessage = ZMQMessage(Seq(Frame(bytes))) + def apply(message: Message): ZMQMessage = ZMQMessage(message.toByteArray) +} + +/** + * Configure this socket to have a linger of the specified value + * + * The linger period determines how long pending messages which have yet to be sent to a peer shall linger + * in memory after a socket is closed, and further affects the termination of the socket's context. + * + * The following outlines the different behaviours: + *
    + *
  • The default value of -1 specifies an infinite linger period. + * Pending messages shall not be discarded after the socket is closed; + * attempting to terminate the socket's context shall block until all pending messages + * have been sent to a peer.
  • + *
  • The value of 0 specifies no linger period. Pending messages shall be discarded immediately when the socket is closed.
  • + *
  • Positive values specify an upper bound for the linger period in milliseconds. + * Pending messages shall not be discarded after the socket is closed; + * attempting to terminate the socket's context shall block until either all pending messages have been sent to a peer, + * or the linger period expires, after which any pending messages shall be discarded.
  • + *
+ * + * @param value The value in milliseconds for the linger option + */ +case class Linger(value: Long) extends SocketOption + +/** + * Gets the linger option @see [[akka.zeromq.Linger]] + */ +object Linger extends SocketOptionQuery + +/** + * Sets the recovery interval for multicast transports using the specified socket. + * The recovery interval determines the maximum time in seconds that a receiver can be absent from a multicast group + * before unrecoverable data loss will occur. + * + * Exercise care when setting large recovery intervals as the data needed for recovery will be held in memory. + * For example, a 1 minute recovery interval at a data rate of 1Gbps requires a 7GB in-memory buffer. + * + * @param value The interval in seconds + */ +case class ReconnectIVL(value: Long) extends SocketOption + +/** + * Gets the recover interval @see [[akka.zeromq.ReconnectIVL]] + */ +object ReconnectIVL extends SocketOptionQuery + +/** + * The [[akka.zeromq.ReconnectIVLMax]] option shall set the maximum reconnection interval for the specified socket. + * This is the maximum period ØMQ shall wait between attempts to reconnect. On each reconnect attempt, + * the previous interval shall be doubled untill [[akka.zeromq.ReconnectIVLMax]] is reached. + * This allows for exponential backoff strategy. Default value means no exponential backoff is performed + * and reconnect interval calculations are only based on [[akka.zeromq.ReconnectIVL]]. + * + * @see [[akka.zeromq.ReconnectIVL]] + * + * This is a ZeroMQ 3.0 option + * + * @param value + */ +case class ReconnectIVLMax(value: Long) extends SocketOption +/** + * Gets the max reconnect IVL + * @see [[akka.zeromq.ReconnectIVLMax]] + */ +object ReconnectIVLMax extends SocketOptionQuery + +/** + * The [[akka.zeromq.Backlog]] option shall set the maximum length of the queue of outstanding peer connections + * for the specified socket; this only applies to connection-oriented transports. For details refer to your + * operating system documentation for the listen function. + * + * @param value + */ +case class Backlog(value: Long) extends SocketOption +/** + * Gets the backlog + * @see [[akka.zeromq.Backlog]] + */ +object Backlog extends SocketOptionQuery + +/** + * Limits the size of the inbound message. + * If a peer sends a message larger than [[akka.zeromq.MaxMsgSize]] it is disconnected. + * Value of -1 means no limit. + * + * This is a ZeroMQ 3.0 option + * + * @param value + */ +case class MaxMsgSize(value: Long) extends SocketOption +object MaxMsgSize extends SocketOptionQuery + +/** + * The [[akka.zeromq.SndHWM]] option shall set the high water mark for outbound messages on the specified socket. + * The high water mark is a hard limit on the maximum number of outstanding messages ØMQ shall queue in memory + * for any single peer that the specified socket is communicating with. + * + * If this limit has been reached the socket shall enter an exceptional state and depending on the socket type, + * ØMQ shall take appropriate action such as blocking or dropping sent messages. + * + * This is a ZeroMQ 3.0 option + * + * @param value + */ +case class SndHWM(value: Long) extends SocketOption + +/** + * Gets the SendHWM + * @see [[akka.zeromq.SndHWM]] + */ +object SndHWM extends SocketOptionQuery + +/** + * The [[akka.zeromq.RcvHWM]] option shall set the high water mark for inbound messages on the specified socket. + * The high water mark is a hard limit on the maximum number of outstanding messages ØMQ shall queue + * in memory for any single peer that the specified socket is communicating with. + * + * If this limit has been reached the socket shall enter an exceptional state and depending on the socket type, + * ØMQ shall take appropriate action such as blocking or dropping sent messages. + * + * This is a ZeroMQ 3.0 option + * + * @param value + */ +case class RcvHWM(value: Long) extends SocketOption + +/** + * Gets the RcvHWM + * @see [[akka.zeromq.RcvHWM]] + */ +object RcvHWM extends SocketOptionQuery + +/** + * The [[akka.zeromq.HWM]] option shall set the high water mark for the specified socket. + * The high water mark is a hard limit on the maximum number of outstanding messages ØMQ shall queue in memory for + * any single peer that the specified socket is communicating with. + * + * If this limit has been reached the socket shall enter an exceptional state and depending on the socket type, + * ØMQ shall take appropriate action such as blocking or dropping sent messages. + * The default [[akka.zeromq.HWM]] value of zero means "no limit". + * + * @param value + */ +case class HWM(value: Long) extends SocketOption + +/** + * The [[akka.zeromq.Swap]] option shall set the disk offload (swap) size for the specified socket. + * A socket which has [[akka.zeromq.Swap]] set to a non-zero value may exceed its high water mark; + * in this case outstanding messages shall be offloaded to storage on disk rather than held in memory. + * + * The value of [[akka.zeromq.Swap]] defines the maximum size of the swap space in bytes. + * + * @param value + */ +case class Swap(value: Long) extends SocketOption + +/** + * Gets the [[akka.zeromq.Swap]] + * + * @see [[akka.zeromq.Swap]] + */ +object Swap extends SocketOptionQuery + +/** + * The [[akka.zeromq.Affinity]] option shall set the I/O thread affinity for newly created connections on the specified socket. + * + * Affinity determines which threads from the ØMQ I/O thread pool associated with the socket's context shall handle + * newly created connections. A value of zero specifies no affinity, meaning that work shall be distributed fairly + * among all ØMQ I/O threads in the thread pool. For non-zero values, the lowest bit corresponds to thread 1, + * second lowest bit to thread 2 and so on. For example, a value of 3 specifies that subsequent connections + * on socket shall be handled exclusively by I/O threads 1 and 2. + * + * @param value + */ +case class Affinity(value: Long) extends SocketOption + +/** + * Gets the [[akka.zeromq.Affinity]] value + */ +object Affinity extends SocketOptionQuery + +/** + * Sets the identity of the specified socket. Socket identity determines if existing ØMQ infrastructure + * (message queues, forwarding devices) shall be identified with a specific application and persist across multiple + * runs of the application. + * + * If the socket has no identity, each run of an application is completely separate from other runs. + * However, with identity set the socket shall re-use any existing ØMQ infrastructure configured by the previous run(s). + * Thus the application may receive messages that were sent in the meantime, message queue limits shall be shared + * with previous run(s) and so on. + * + * Identity should be at least one byte and at most 255 bytes long. + * Identities starting with binary zero are reserved for use by ØMQ infrastructure. + * + * @param value The identity string for this socket + */ +case class Identity(value: Array[Byte]) extends SocketOption + +/** + * Gets the [[akka.zeromq.Identity]] value + */ +object Identity extends SocketOptionQuery + +/** + * Sets the maximum send or receive data rate for multicast transports such as pgm using the specified socket. + * + * @param value The kilobits per second + */ +case class Rate(value: Long) extends SocketOption + +/** + * Gets the send or receive rate for the socket + */ +object Rate extends SocketOptionQuery + +/** + * Sets the recovery interval for multicast transports using the specified socket. + * The recovery interval determines the maximum time in seconds that a receiver can be absent from a multicast group + * before unrecoverable data loss will occur. + * + * Exercise care when setting large recovery intervals as the data needed for recovery will be held in memory. + * For example, a 1 minute recovery interval at a data rate of 1Gbps requires a 7GB in-memory buffer. + * + * @param value The interval in seconds + */ +case class RecoveryInterval(value: Long) extends SocketOption + +/** + * Gets the [[akka.zeromq.RecoveryInterval]] + */ +object RecoveryInterval extends SocketOptionQuery + +/** + * Controls whether data sent via multicast transports using the specified socket can also be received by the sending + * host via loop-back. A value of zero disables the loop-back functionality, while the default value of 1 enables the + * loop-back functionality. Leaving multicast loop-back enabled when it is not required can have a negative impact + * on performance. Where possible, disable McastLoop in production environments. + * + * @param value Flag indicating whether or not loopback multicast is enabled + */ +case class MulticastLoop(value: Boolean) extends SocketOption + +/** + * Gets the [[akka.zeromq.MulticastLoop]] + */ +object MulticastLoop extends SocketOptionQuery + +/** + * Sets the time-to-live field in every multicast packet sent from this socket. + * The default is 1 which means that the multicast packets don't leave the local network. + * + * This is za ZeroMQ 3.0 option + * + * @param value + */ +case class MulticastHops(value: Long) extends SocketOption + +/** + * Gets the [[akka.zeromq.MulticastHops]] + */ +object MulticastHops extends SocketOptionQuery + +/** + * The [[akka.zeromq.SendBufferSize]] option shall set the underlying kernel transmit buffer size for the socket to + * the specified size in bytes. A value of zero means leave the OS default unchanged. + * For details please refer to your operating system documentation for the SO_SNDBUF socket option. + * + * This is a ZeroMQ 2.x only option + * + * @param value + */ +case class SendBufferSize(value: Long) extends SocketOption + +/** + * Gets the [[akka.zeromq.SendBufferSize]] + */ +object SendBufferSize extends SocketOptionQuery + +/** + * The [[akka.zeromq.ReceiveBufferSize]] option shall set the underlying kernel receive buffer size for the socket to + * the specified size in bytes. A value of zero means leave the OS default unchanged. + * For details refer to your operating system documentation for the SO_RCVBUF socket option. + * @param value + */ +case class ReceiveBufferSize(value: Long) extends SocketOption + +/** + * Gets the [[akka.zeromq.ReceiveBufferSize]] + */ +object ReceiveBufferSize extends SocketOptionQuery + +/** + * Gets the file descriptor associated with the ZeroMQ socket + */ +object FileDescriptor extends SocketOptionQuery diff --git a/akka-zeromq/src/main/scala/akka/zeromq/Deserializer.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZMQMessageDeserializer.scala similarity index 67% rename from akka-zeromq/src/main/scala/akka/zeromq/Deserializer.scala rename to akka-zeromq/src/main/scala/akka/zeromq/ZMQMessageDeserializer.scala index b9913ea560..7bddc08bea 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/Deserializer.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZMQMessageDeserializer.scala @@ -3,9 +3,16 @@ */ package akka.zeromq +/** + * A single message frame of a zeromq message + * @param payload + */ case class Frame(payload: Seq[Byte]) object Frame { def apply(s: String): Frame = Frame(s.getBytes) } +/** + * Deserializes ZeroMQ messages into an immutable sequence of frames + */ class ZMQMessageDeserializer extends Deserializer { def apply(frames: Seq[Frame]) = ZMQMessage(frames) } diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala index 54d32c3264..ffc6537867 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala @@ -6,12 +6,21 @@ package akka.zeromq import akka.util.duration._ import org.zeromq.{ ZMQ ⇒ JZMQ } import akka.actor._ -import akka.dispatch.{ Dispatcher, Await } +import akka.dispatch.{ Await } +/** + * A Model to represent a version of the zeromq library + * @param major + * @param minor + * @param patch + */ case class ZeroMQVersion(major: Int, minor: Int, patch: Int) { override def toString = "%d.%d.%d".format(major, minor, patch) } +/** + * The [[akka.actor.ExtensionId]] and [[akka.actor.ExtensionIdProvider]] for the ZeroMQ module + */ object ZeroMQExtension extends ExtensionId[ZeroMQExtension] with ExtensionIdProvider { def lookup() = this def createExtension(system: ActorSystemImpl) = new ZeroMQExtension(system) @@ -25,25 +34,49 @@ object ZeroMQExtension extends ExtensionId[ZeroMQExtension] with ExtensionIdProv } } } + +/** + * The extension for the ZeroMQ module + * + * @param system The ActorSystem this extension belongs to. + */ class ZeroMQExtension(system: ActorSystem) extends Extension { + /** + * The version of the ZeroMQ library + * @return a [[akka.zeromq.ZeroMQVersion]] + */ def version = { ZeroMQVersion(JZMQ.getMajorVersion, JZMQ.getMinorVersion, JZMQ.getPatchVersion) } + /** + * Factory method to create the [[akka.actor.Props]] to build the ZeroMQ socket actor. + * + * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socket + * @return the [[akka.actor.Props]] + */ def newSocketProps(socketParameters: SocketOption*): Props = { verifyZeroMQVersion require(ZeroMQExtension.check[SocketType.ZMQSocketType](socketParameters), "A socket type is required") Props(new ConcurrentSocketActor(socketParameters)).withDispatcher("akka.zeromq.socket-dispatcher") } + /** + * Factory method to create the actor representing the ZeroMQ socket. + * You can pass in as many configuration options as you want and the order of the configuration options doesn't matter + * They are matched on type and the first one found wins. + * + * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socke + * @return + */ def newSocket(socketParameters: SocketOption*): ActorRef = { implicit val timeout = system.settings.ActorTimeout - val req = (zeromq ? newSocketProps(socketParameters: _*)).mapTo[ActorRef] + val req = (zeromqGuardian ? newSocketProps(socketParameters: _*)).mapTo[ActorRef] Await.result(req, timeout.duration) } - val zeromq: ActorRef = { + private val zeromqGuardian: ActorRef = { verifyZeroMQVersion system.actorOf(Props(new Actor { def receive = { case p: Props ⇒ sender ! context.actorOf(p) } diff --git a/akka-zeromq/src/main/scala/package.scala b/akka-zeromq/src/main/scala/package.scala index e62a16738c..11a62b3540 100644 --- a/akka-zeromq/src/main/scala/package.scala +++ b/akka-zeromq/src/main/scala/package.scala @@ -2,10 +2,25 @@ package akka import actor.ActorSystem +/** + * A package object with an implicit conversion for the actor system as a convenience + */ package object zeromq { - implicit def zeromqSystem(system: ActorSystem) = new { - def zeromq = system.extension(ZeroMQExtension) - } + /** + * Creates a zeromq actor system implicitly + * @param system + * @return An augmented [[akka.actor.ActorSystem]] + */ + implicit def zeromqSystem(system: ActorSystem) = system.extension(ZeroMQExtension) + + /** + * Convenience accessor to subscribe to all events + */ val SubscribeAll = Subscribe(Seq.empty) + + /** + * Set the linger to 0, doesn't block and discards messages that haven't been sent yet. + */ + val NoLinger = Linger(0) } \ No newline at end of file From ddadcbbbd05bb36dd127758bd08b4608aa47bd20 Mon Sep 17 00:00:00 2001 From: Ivan Porto Carrero Date: Fri, 20 Jan 2012 00:36:33 +0100 Subject: [PATCH 061/152] Throw a clearer error message when the socket handle is null --- .../src/main/scala/akka/zeromq/ConcurrentSocketActor.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala index 3f6caad411..1540ec0ec0 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala @@ -142,7 +142,9 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A private def socketFromParams() = { require(ZeroMQExtension.check[SocketType.ZMQSocketType](params), "A socket type is required") - (params collectFirst { case t: SocketType.ZMQSocketType ⇒ zmqContext.socket(t) } get) + (params + collectFirst { case t: SocketType.ZMQSocketType ⇒ zmqContext.socket(t) } + getOrElse (throw new NoSocketHandleException)) } private def deserializerFromParams = { From 973893be8679b0106eadf2447d7656c3cbb38a0f Mon Sep 17 00:00:00 2001 From: Ivan Porto Carrero Date: Fri, 20 Jan 2012 01:16:24 +0100 Subject: [PATCH 062/152] Rename HWM options, remove Frame companion object --- .../akka/zeromq/ConcurrentSocketActor.scala | 68 +++++++++---------- .../main/scala/akka/zeromq/SocketOption.scala | 24 +++---- .../akka/zeromq/ZMQMessageDeserializer.scala | 1 - 3 files changed, 46 insertions(+), 47 deletions(-) diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala index 1540ec0ec0..ad6cf0ff72 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala @@ -71,40 +71,40 @@ private[zeromq] class ConcurrentSocketActor(params: Seq[SocketOption]) extends A } private def handleSocketOption: Receive = { - case Linger(value) ⇒ socket.setLinger(value) - case ReconnectIVL(value) ⇒ socket.setReconnectIVL(value) - case Backlog(value) ⇒ socket.setBacklog(value) - case ReconnectIVLMax(value) ⇒ socket.setReconnectIVLMax(value) - case MaxMsgSize(value) ⇒ socket.setMaxMsgSize(value) - case SndHWM(value) ⇒ socket.setSndHWM(value) - case RcvHWM(value) ⇒ socket.setRcvHWM(value) - case HWM(value) ⇒ socket.setHWM(value) - case Swap(value) ⇒ socket.setSwap(value) - case Affinity(value) ⇒ socket.setAffinity(value) - case Identity(value) ⇒ socket.setIdentity(value) - case Rate(value) ⇒ socket.setRate(value) - case RecoveryInterval(value) ⇒ socket.setRecoveryInterval(value) - case MulticastLoop(value) ⇒ socket.setMulticastLoop(value) - case MulticastHops(value) ⇒ socket.setMulticastHops(value) - case SendBufferSize(value) ⇒ socket.setSendBufferSize(value) - case ReceiveBufferSize(value) ⇒ socket.setReceiveBufferSize(value) - case Linger ⇒ sender ! socket.getLinger - case ReconnectIVL ⇒ sender ! socket.getReconnectIVL - case Backlog ⇒ sender ! socket.getBacklog - case ReconnectIVLMax ⇒ sender ! socket.getReconnectIVLMax - case MaxMsgSize ⇒ sender ! socket.getMaxMsgSize - case SndHWM ⇒ sender ! socket.getSndHWM - case RcvHWM ⇒ sender ! socket.getRcvHWM - case Swap ⇒ sender ! socket.getSwap - case Affinity ⇒ sender ! socket.getAffinity - case Identity ⇒ sender ! socket.getIdentity - case Rate ⇒ sender ! socket.getRate - case RecoveryInterval ⇒ sender ! socket.getRecoveryInterval - case MulticastLoop ⇒ sender ! socket.hasMulticastLoop - case MulticastHops ⇒ sender ! socket.getMulticastHops - case SendBufferSize ⇒ sender ! socket.getSendBufferSize - case ReceiveBufferSize ⇒ sender ! socket.getReceiveBufferSize - case FileDescriptor ⇒ sender ! socket.getFD + case Linger(value) ⇒ socket.setLinger(value) + case ReconnectIVL(value) ⇒ socket.setReconnectIVL(value) + case Backlog(value) ⇒ socket.setBacklog(value) + case ReconnectIVLMax(value) ⇒ socket.setReconnectIVLMax(value) + case MaxMsgSize(value) ⇒ socket.setMaxMsgSize(value) + case SendHighWatermark(value) ⇒ socket.setSndHWM(value) + case ReceiveHighWatermark(value) ⇒ socket.setRcvHWM(value) + case HighWatermark(value) ⇒ socket.setHWM(value) + case Swap(value) ⇒ socket.setSwap(value) + case Affinity(value) ⇒ socket.setAffinity(value) + case Identity(value) ⇒ socket.setIdentity(value) + case Rate(value) ⇒ socket.setRate(value) + case RecoveryInterval(value) ⇒ socket.setRecoveryInterval(value) + case MulticastLoop(value) ⇒ socket.setMulticastLoop(value) + case MulticastHops(value) ⇒ socket.setMulticastHops(value) + case SendBufferSize(value) ⇒ socket.setSendBufferSize(value) + case ReceiveBufferSize(value) ⇒ socket.setReceiveBufferSize(value) + case Linger ⇒ sender ! socket.getLinger + case ReconnectIVL ⇒ sender ! socket.getReconnectIVL + case Backlog ⇒ sender ! socket.getBacklog + case ReconnectIVLMax ⇒ sender ! socket.getReconnectIVLMax + case MaxMsgSize ⇒ sender ! socket.getMaxMsgSize + case SendHighWatermark ⇒ sender ! socket.getSndHWM + case ReceiveHighWatermark ⇒ sender ! socket.getRcvHWM + case Swap ⇒ sender ! socket.getSwap + case Affinity ⇒ sender ! socket.getAffinity + case Identity ⇒ sender ! socket.getIdentity + case Rate ⇒ sender ! socket.getRate + case RecoveryInterval ⇒ sender ! socket.getRecoveryInterval + case MulticastLoop ⇒ sender ! socket.hasMulticastLoop + case MulticastHops ⇒ sender ! socket.getMulticastHops + case SendBufferSize ⇒ sender ! socket.getSendBufferSize + case ReceiveBufferSize ⇒ sender ! socket.getReceiveBufferSize + case FileDescriptor ⇒ sender ! socket.getFD } private def internalMessage: Receive = { diff --git a/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala b/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala index 457129ea5c..ad01e90d47 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala @@ -305,7 +305,7 @@ case class MaxMsgSize(value: Long) extends SocketOption object MaxMsgSize extends SocketOptionQuery /** - * The [[akka.zeromq.SndHWM]] option shall set the high water mark for outbound messages on the specified socket. + * The [[akka.zeromq.SendHighWatermark]] option shall set the high water mark for outbound messages on the specified socket. * The high water mark is a hard limit on the maximum number of outstanding messages ØMQ shall queue in memory * for any single peer that the specified socket is communicating with. * @@ -316,16 +316,16 @@ object MaxMsgSize extends SocketOptionQuery * * @param value */ -case class SndHWM(value: Long) extends SocketOption +case class SendHighWatermark(value: Long) extends SocketOption /** * Gets the SendHWM - * @see [[akka.zeromq.SndHWM]] + * @see [[akka.zeromq.SendHighWatermark]] */ -object SndHWM extends SocketOptionQuery +object SendHighWatermark extends SocketOptionQuery /** - * The [[akka.zeromq.RcvHWM]] option shall set the high water mark for inbound messages on the specified socket. + * The [[akka.zeromq.ReceiveHighWatermark]] option shall set the high water mark for inbound messages on the specified socket. * The high water mark is a hard limit on the maximum number of outstanding messages ØMQ shall queue * in memory for any single peer that the specified socket is communicating with. * @@ -336,26 +336,26 @@ object SndHWM extends SocketOptionQuery * * @param value */ -case class RcvHWM(value: Long) extends SocketOption +case class ReceiveHighWatermark(value: Long) extends SocketOption /** - * Gets the RcvHWM - * @see [[akka.zeromq.RcvHWM]] + * Gets the ReceiveHighWatermark + * @see [[akka.zeromq.ReceiveHighWatermark]] */ -object RcvHWM extends SocketOptionQuery +object ReceiveHighWatermark extends SocketOptionQuery /** - * The [[akka.zeromq.HWM]] option shall set the high water mark for the specified socket. + * The [[akka.zeromq.HighWatermark]] option shall set the high water mark for the specified socket. * The high water mark is a hard limit on the maximum number of outstanding messages ØMQ shall queue in memory for * any single peer that the specified socket is communicating with. * * If this limit has been reached the socket shall enter an exceptional state and depending on the socket type, * ØMQ shall take appropriate action such as blocking or dropping sent messages. - * The default [[akka.zeromq.HWM]] value of zero means "no limit". + * The default [[akka.zeromq.HighWatermark]] value of zero means "no limit". * * @param value */ -case class HWM(value: Long) extends SocketOption +case class HighWatermark(value: Long) extends SocketOption /** * The [[akka.zeromq.Swap]] option shall set the disk offload (swap) size for the specified socket. diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ZMQMessageDeserializer.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZMQMessageDeserializer.scala index 7bddc08bea..3acef255a7 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ZMQMessageDeserializer.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZMQMessageDeserializer.scala @@ -8,7 +8,6 @@ package akka.zeromq * @param payload */ case class Frame(payload: Seq[Byte]) -object Frame { def apply(s: String): Frame = Frame(s.getBytes) } /** * Deserializes ZeroMQ messages into an immutable sequence of frames From 7a3df2f2d811a98faa4a7c8fe728644eb24ad1a6 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 20 Jan 2012 08:35:23 +0100 Subject: [PATCH 063/152] minor cleanup --- akka-docs/project/migration-guide-1.3.x-2.0.x.rst | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst index 751535feba..a3f874894d 100644 --- a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst +++ b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst @@ -10,15 +10,10 @@ :local: :depth: 3 -Actors -====== - The 2.0 release contains several new features which require source-level changes in client code. This API cleanup is planned to be the last one for a significant amount of time. -Detailed migration guide will be written. - Migration Kit ============= @@ -472,7 +467,7 @@ The dispatcher is assigned to the actor in a different way. v1.3:: actorRef.dispatcher = MyGlobals.myDispatcher - slef.dispatcher = = MyGlobals.myDispatcher + self.dispatcher = MyGlobals.myDispatcher v2.0:: From 56bd8a6882808082391d29317e912c8ce1d2c98b Mon Sep 17 00:00:00 2001 From: Ivan Porto Carrero Date: Fri, 20 Jan 2012 09:27:23 +0100 Subject: [PATCH 064/152] Adds missing docs and/or return types --- akka-docs/scala/zeromq.rst | 4 +--- .../src/main/scala/akka/zeromq/Response.scala | 18 +++++++++++++++ .../main/scala/akka/zeromq/Responses.scala | 8 ------- .../main/scala/akka/zeromq/SocketOption.scala | 22 +++++++++---------- 4 files changed, 29 insertions(+), 23 deletions(-) create mode 100644 akka-zeromq/src/main/scala/akka/zeromq/Response.scala delete mode 100644 akka-zeromq/src/main/scala/akka/zeromq/Responses.scala diff --git a/akka-docs/scala/zeromq.rst b/akka-docs/scala/zeromq.rst index b21c81d9c4..e080979910 100644 --- a/akka-docs/scala/zeromq.rst +++ b/akka-docs/scala/zeromq.rst @@ -8,8 +8,6 @@ ZeroMQ .. contents:: :local: -Module stability: **IN PROGRESS** - Akka provides a ZeroMQ module which abstracts a ZeroMQ connection and therefore allows interaction between Akka actors to take place over ZeroMQ connections. The messages can be of a proprietary format or they can be defined using Protobuf. The socket actor is fault-tolerant by default and when you use the newSocket method to create new sockets it will properly reinitialize the socket. ZeroMQ is very opinionated when it comes to multi-threading so configuration option `akka.zeromq.socket-dispatcher` always needs to be configured to a PinnedDispatcher, because the actual ZeroMQ socket can only be accessed by the thread that created it. @@ -29,7 +27,7 @@ ZeroMQ supports multiple connectivity patterns, each aimed to meet a different s will create a ZeroMQ Publisher socket that is Bound to the port 1234 on localhost. Importing the akka.zeromq._ package ensures that the implicit zeromq method is available. -Similarly you can create a subscribtion socket, that subscribes to all messages from the publisher using: +Similarly you can create a subscription socket, that subscribes to all messages from the publisher using: .. code-block:: scala diff --git a/akka-zeromq/src/main/scala/akka/zeromq/Response.scala b/akka-zeromq/src/main/scala/akka/zeromq/Response.scala new file mode 100644 index 0000000000..f916ba0a4f --- /dev/null +++ b/akka-zeromq/src/main/scala/akka/zeromq/Response.scala @@ -0,0 +1,18 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ +package akka.zeromq + +/** + * Base trait for the events raised by a ZeroMQ socket actor + */ +sealed trait Response + +/** + * When the ZeroMQ socket connects it sends this message to a listener + */ +case object Connecting extends Response +/** + * When the ZeroMQ socket disconnects it sends this message to a listener + */ +case object Closed extends Response diff --git a/akka-zeromq/src/main/scala/akka/zeromq/Responses.scala b/akka-zeromq/src/main/scala/akka/zeromq/Responses.scala deleted file mode 100644 index 43200a959c..0000000000 --- a/akka-zeromq/src/main/scala/akka/zeromq/Responses.scala +++ /dev/null @@ -1,8 +0,0 @@ -/** - * Copyright (C) 2009-2011 Typesafe Inc. - */ -package akka.zeromq - -sealed trait Response -case object Connecting extends Response -case object Closed extends Response diff --git a/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala b/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala index ad01e90d47..41d7cf867c 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala @@ -8,6 +8,7 @@ import org.zeromq.{ ZMQ ⇒ JZMQ } import akka.actor.ActorRef import akka.util.duration._ import akka.util.Duration +import org.zeromq.ZMQ.{Poller, Socket} /** * Marker trait representing request messages for zeromq @@ -55,27 +56,24 @@ case class Connect(endpoint: String) extends SocketConnectOption * Companion object for a ZeroMQ I/O thread pool */ object Context { - def apply(numIoThreads: Int = 1) = new Context(numIoThreads) + def apply(numIoThreads: Int = 1): Context = new Context(numIoThreads) } /** * Represents an I/O thread pool for ZeroMQ sockets. - * By default the ZeroMQ module uses a thread pool with 1 thread. For most applications that should be - * sufficient + * By default the ZeroMQ module uses an I/O thread pool with 1 thread. + * For most applications that should be sufficient * * @param numIoThreads */ class Context(numIoThreads: Int) extends SocketMeta { private val context = JZMQ.context(numIoThreads) - def socket(socketType: SocketType.ZMQSocketType) = { - context.socket(socketType.id) - } - def poller = { - context.poller - } - def term = { - context.term - } + + def socket(socketType: SocketType.ZMQSocketType): Socket = context.socket(socketType.id) + + def poller: Poller = context.poller + + def term: Unit = context.term } /** From 05e4af750081aeb757dab23c5a9885949ad901ea Mon Sep 17 00:00:00 2001 From: Ivan Porto Carrero Date: Fri, 20 Jan 2012 10:26:55 +0100 Subject: [PATCH 065/152] Adds java api helpers --- .../src/main/java/akka/zeromq/ZeroMQ.java | 260 ++++++++++++++++++ .../main/scala/akka/zeromq/SocketOption.scala | 2 +- .../scala/akka/zeromq/ZeroMQExtension.scala | 154 ++++++++++- 3 files changed, 414 insertions(+), 2 deletions(-) create mode 100644 akka-zeromq/src/main/java/akka/zeromq/ZeroMQ.java diff --git a/akka-zeromq/src/main/java/akka/zeromq/ZeroMQ.java b/akka-zeromq/src/main/java/akka/zeromq/ZeroMQ.java new file mode 100644 index 0000000000..e539af18af --- /dev/null +++ b/akka-zeromq/src/main/java/akka/zeromq/ZeroMQ.java @@ -0,0 +1,260 @@ +package akka.zeromq; + +/** + * Java API for akka.zeromq + */ +public class ZeroMQ { + + /** + * The message that is sent when an ZeroMQ socket connects. + *

+ *

+     * if (message == connecting()) {
+     *   // Socket connected
+     * }
+     * 
+ * + * @return the single instance of Connecting + */ + public final static Connecting$ connecting() { + return Connecting$.MODULE$; + } + + /** + * The message that is sent when an ZeroMQ socket disconnects. + *

+ *

+     * if (message == closed()) {
+     *   // Socket disconnected
+     * }
+     * 
+ * + * @return the single instance of Closed + */ + public final static Closed$ closed() { + return Closed$.MODULE$; + } + + /** + * The message to ask a ZeroMQ socket for its affinity configuration. + *

+ *

+     * socket.ask(affinity())
+     * 
+ * + * @return the single instance of Affinity + */ + public final static Affinity$ affinity() { + return Affinity$.MODULE$; + } + + /** + * The message to ask a ZeroMQ socket for its backlog configuration. + *

+ *

+     * socket.ask(backlog())
+     * 
+ * + * @return the single instance of Backlog + */ + public final static Backlog$ backlog() { + return Backlog$.MODULE$; + } + + /** + * The message to ask a ZeroMQ socket for its file descriptor configuration. + *

+ *

+     * socket.ask(fileDescriptor())
+     * 
+ * + * @return the single instance of FileDescriptor + */ + public final static FileDescriptor$ fileDescriptor() { + return FileDescriptor$.MODULE$; + } + + /** + * The message to ask a ZeroMQ socket for its identity configuration. + *

+ *

+     * socket.ask(identity())
+     * 
+ * + * @return the single instance of Identity + */ + public final static Identity$ identity() { + return Identity$.MODULE$; + } + + /** + * The message to ask a ZeroMQ socket for its linger configuration. + *

+ *

+     * socket.ask(linger())
+     * 
+ * + * @return the single instance of Linger + */ + public final static Linger$ linger() { + return Linger$.MODULE$; + } + + /** + * The message to ask a ZeroMQ socket for its max message size configuration. + *

+ *

+     * socket.ask(maxMessageSize())
+     * 
+ * + * @return the single instance of MaxMsgSize + */ + public final static MaxMsgSize$ maxMessageSize() { + return MaxMsgSize$.MODULE$; + } + + /** + * The message to ask a ZeroMQ socket for its multicast hops configuration. + *

+ *

+     * socket.ask(multicastHops())
+     * 
+ * + * @return the single instance of MulticastHops + */ + public final static MulticastHops$ multicastHops() { + return MulticastHops$.MODULE$; + } + + /** + * The message to ask a ZeroMQ socket for its multicast loop configuration. + *

+ *

+     * socket.ask(multicastLoop())
+     * 
+ * + * @return the single instance of MulticastLoop + */ + public final static MulticastLoop$ multicastLoop() { + return MulticastLoop$.MODULE$; + } + + /** + * The message to ask a ZeroMQ socket for its rate configuration. + *

+ *

+     * socket.ask(rate())
+     * 
+ * + * @return the single instance of Rate + */ + public final static Rate$ rate() { + return Rate$.MODULE$; + } + + /** + * The message to ask a ZeroMQ socket for its receive bufferSize configuration. + *

+ *

+     * socket.ask(receiveBufferSize())
+     * 
+ * + * @return the single instance of ReceiveBufferSize + */ + public final static ReceiveBufferSize$ receiveBufferSize() { + return ReceiveBufferSize$.MODULE$; + } + + /** + * The message to ask a ZeroMQ socket for its receive high watermark configuration. + *

+ *

+     * socket.ask(receiveHighWatermark())
+     * 
+ * + * @return the single instance of ReceiveHighWatermark + */ + public final static ReceiveHighWatermark$ receiveHighWatermark() { + return ReceiveHighWatermark$.MODULE$; + } + + /** + * The message to ask a ZeroMQ socket for its reconnect interval configuration. + *

+ *

+     * socket.ask(reconnectIVL())
+     * 
+ * + * @return the single instance of ReconnectIVL + */ + public final static ReconnectIVL$ reconnectIVL() { + return ReconnectIVL$.MODULE$; + } + + /** + * The message to ask a ZeroMQ socket for its max reconnect interval configuration. + *

+ *

+     * socket.ask(reconnectIVLMax())
+     * 
+ * + * @return the single instance of ReconnectIVLMax + */ + public final static ReconnectIVLMax$ reconnectIVLMax() { + return ReconnectIVLMax$.MODULE$; + } + + /** + * The message to ask a ZeroMQ socket for its recovery interval configuration. + *

+ *

+     * socket.ask(recoveryInterval())
+     * 
+ * + * @return the single instance of RecoveryInterval + */ + public final static RecoveryInterval$ recoveryInterval() { + return RecoveryInterval$.MODULE$; + } + + /** + * The message to ask a ZeroMQ socket for its send buffer size configuration. + *

+ *

+     * socket.ask(sendBufferSize())
+     * 
+ * + * @return the single instance of SendBufferSize + */ + public final static SendBufferSize$ sendBufferSize() { + return SendBufferSize$.MODULE$; + } + + /** + * The message to ask a ZeroMQ socket for its send high watermark configuration. + *

+ *

+     * socket.ask(sendHighWatermark())
+     * 
+ * + * @return the single instance of SendHighWatermark + */ + public final static SendHighWatermark$ sendHighWatermark() { + return SendHighWatermark$.MODULE$; + } + + /** + * The message to ask a ZeroMQ socket for its swap configuration. + *

+ *

+     * socket.ask(swap())
+     * 
+ * + * @return the single instance of Swap + */ + public final static Swap$ swap() { + return Swap$.MODULE$; + } + + +} diff --git a/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala b/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala index 41d7cf867c..488b99fa22 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala @@ -8,7 +8,7 @@ import org.zeromq.{ ZMQ ⇒ JZMQ } import akka.actor.ActorRef import akka.util.duration._ import akka.util.Duration -import org.zeromq.ZMQ.{Poller, Socket} +import org.zeromq.ZMQ.{ Poller, Socket } /** * Marker trait representing request messages for zeromq diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala index ffc6537867..68adc998d5 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala @@ -62,13 +62,77 @@ class ZeroMQExtension(system: ActorSystem) extends Extension { Props(new ConcurrentSocketActor(socketParameters)).withDispatcher("akka.zeromq.socket-dispatcher") } + /** + * Java API helper + * Factory method to create the [[akka.actor.Props]] to build a ZeroMQ Publisher socket actor. + * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socket + * @return the [[akka.actor.Props]] + */ + def newPubSocketProps(socketParameters: SocketOption*): Props = newSocketProps((SocketType.Pub +: socketParameters): _*) + + /** + * Java API helper + * Factory method to create the [[akka.actor.Props]] to build a ZeroMQ Subscriber socket actor. + * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socket + * @return the [[akka.actor.Props]] + */ + def newSubSocketProps(socketParameters: SocketOption*): Props = newSocketProps((SocketType.Sub +: socketParameters): _*) + + /** + * Java API helper + * Factory method to create the [[akka.actor.Props]] to build a ZeroMQ Dealer socket actor. + * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socket + * @return the [[akka.actor.Props]] + */ + def newDealerSocketProps(socketParameters: SocketOption*): Props = newSocketProps((SocketType.Dealer +: socketParameters): _*) + + /** + * Java API helper + * Factory method to create the [[akka.actor.Props]] to build a ZeroMQ Router socket actor. + * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socket + * @return the [[akka.actor.Props]] + */ + def newRouterSocketProps(socketParameters: SocketOption*): Props = newSocketProps((SocketType.Router +: socketParameters): _*) + + /** + * Java API helper + * Factory method to create the [[akka.actor.Props]] to build a ZeroMQ Push socket actor. + * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socket + * @return the [[akka.actor.Props]] + */ + def newPushSocketProps(socketParameters: SocketOption*): Props = newSocketProps((SocketType.Push +: socketParameters): _*) + + /** + * Java API helper + * Factory method to create the [[akka.actor.Props]] to build a ZeroMQ Pull socket actor. + * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socket + * @return the [[akka.actor.Props]] + */ + def newPullSocketProps(socketParameters: SocketOption*): Props = newSocketProps((SocketType.Pull +: socketParameters): _*) + + /** + * Java API helper + * Factory method to create the [[akka.actor.Props]] to build a ZeroMQ Req socket actor. + * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socket + * @return the [[akka.actor.Props]] + */ + def newReqSocketProps(socketParameters: SocketOption*): Props = newSocketProps((SocketType.Rep +: socketParameters): _*) + + /** + * Java API helper + * Factory method to create the [[akka.actor.Props]] to build a ZeroMQ Rep socket actor. + * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socket + * @return the [[akka.actor.Props]] + */ + def newRepSocketProps(socketParameters: SocketOption*): Props = newSocketProps((SocketType.Req +: socketParameters): _*) + /** * Factory method to create the actor representing the ZeroMQ socket. * You can pass in as many configuration options as you want and the order of the configuration options doesn't matter * They are matched on type and the first one found wins. * * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socke - * @return + * @return the [[akka.actor.ActorRef]] */ def newSocket(socketParameters: SocketOption*): ActorRef = { implicit val timeout = system.settings.ActorTimeout @@ -76,6 +140,94 @@ class ZeroMQExtension(system: ActorSystem) extends Extension { Await.result(req, timeout.duration) } + /** + * Java API helper + * Factory method to create the actor representing the ZeroMQ Publisher socket. + * You can pass in as many configuration options as you want and the order of the configuration options doesn't matter + * They are matched on type and the first one found wins. + * + * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socke + * @return the [[akka.actor.ActorRef]] + */ + def newPubSocket(socketParameters: SocketOption*): ActorRef = newSocket((SocketType.Pub +: socketParameters): _*) + + /** + * Java API helper + * Factory method to create the actor representing the ZeroMQ Subscriber socket. + * You can pass in as many configuration options as you want and the order of the configuration options doesn't matter + * They are matched on type and the first one found wins. + * + * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socke + * @return the [[akka.actor.ActorRef]] + */ + def newSubSocket(socketParameters: SocketOption*): ActorRef = newSocket((SocketType.Sub +: socketParameters): _*) + + /** + * Java API helper + * Factory method to create the actor representing the ZeroMQ Dealer socket. + * You can pass in as many configuration options as you want and the order of the configuration options doesn't matter + * They are matched on type and the first one found wins. + * + * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socke + * @return the [[akka.actor.ActorRef]] + */ + def newDealerSocket(socketParameters: SocketOption*): ActorRef = newSocket((SocketType.Dealer +: socketParameters): _*) + + /** + * Java API helper + * Factory method to create the actor representing the ZeroMQ Router socket. + * You can pass in as many configuration options as you want and the order of the configuration options doesn't matter + * They are matched on type and the first one found wins. + * + * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socke + * @return the [[akka.actor.ActorRef]] + */ + def newRouterSocket(socketParameters: SocketOption*): ActorRef = newSocket((SocketType.Router +: socketParameters): _*) + + /** + * Java API helper + * Factory method to create the actor representing the ZeroMQ Push socket. + * You can pass in as many configuration options as you want and the order of the configuration options doesn't matter + * They are matched on type and the first one found wins. + * + * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socke + * @return the [[akka.actor.ActorRef]] + */ + def newPushSocket(socketParameters: SocketOption*): ActorRef = newSocket((SocketType.Push +: socketParameters): _*) + + /** + * Java API helper + * Factory method to create the actor representing the ZeroMQ Pull socket. + * You can pass in as many configuration options as you want and the order of the configuration options doesn't matter + * They are matched on type and the first one found wins. + * + * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socke + * @return the [[akka.actor.ActorRef]] + */ + def newPullSocket(socketParameters: SocketOption*): ActorRef = newSocket((SocketType.Pull +: socketParameters): _*) + + /** + * Java API helper + * Factory method to create the actor representing the ZeroMQ Req socket. + * You can pass in as many configuration options as you want and the order of the configuration options doesn't matter + * They are matched on type and the first one found wins. + * + * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socke + * @return the [[akka.actor.ActorRef]] + */ + def newReqSocket(socketParameters: SocketOption*): ActorRef = newSocket((SocketType.Req +: socketParameters): _*) + + /** + * Java API helper + * Factory method to create the actor representing the ZeroMQ Rep socket. + * You can pass in as many configuration options as you want and the order of the configuration options doesn't matter + * They are matched on type and the first one found wins. + * + * @param socketParameters a varargs list of [[akka.zeromq.SocketOption]] to configure the socke + * @return the [[akka.actor.ActorRef]] + */ + def newRepSocket(socketParameters: SocketOption*): ActorRef = newSocket((SocketType.Rep +: socketParameters): _*) + private val zeromqGuardian: ActorRef = { verifyZeroMQVersion system.actorOf(Props(new Actor { From 05d4a6ed38e91954f29a4c4af7c745b8ae1cd084 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 20 Jan 2012 11:30:33 +0100 Subject: [PATCH 066/152] DOC: Extension sample for application specific settings. See #1702 * Added sample and documentation for how to create extension for application specific config * Simplified java usage of ExtensionIdProvider, more aligned with how it is done in scala --- .../test/java/akka/actor/JavaExtension.java | 26 +++--- akka-docs/general/configuration.rst | 17 +++- .../docs/extension/ExtensionDocTestBase.java | 80 ++++++++--------- .../extension/SettingsExtensionDocTest.scala | 8 ++ .../SettingsExtensionDocTestBase.java | 85 +++++++++++++++++++ akka-docs/java/extending-akka.rst | 42 ++++++--- .../docs/extension/ExtensionDocSpec.scala | 75 ++++++++-------- .../extension/SettingsExtensionDocSpec.scala | 77 +++++++++++++++++ akka-docs/scala/extending-akka.rst | 42 +++++++-- akka-docs/scala/testing.rst | 6 +- akka-docs/scala/testkit-example.rst | 6 +- 11 files changed, 345 insertions(+), 119 deletions(-) create mode 100644 akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTest.scala create mode 100644 akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTestBase.java create mode 100644 akka-docs/scala/code/akka/docs/extension/SettingsExtensionDocSpec.scala diff --git a/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java b/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java index dfb3e9230e..734e6fa506 100644 --- a/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java +++ b/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java @@ -16,15 +16,13 @@ import static org.junit.Assert.*; public class JavaExtension { - static class Provider implements ExtensionIdProvider { + static class TestExtensionId extends AbstractExtensionId implements ExtensionIdProvider { + public final static TestExtensionId instance = new TestExtensionId(); + public ExtensionId lookup() { - return defaultInstance; + return instance; } - } - public final static TestExtensionId defaultInstance = new TestExtensionId(); - - static class TestExtensionId extends AbstractExtensionId { public TestExtension createExtension(ActorSystemImpl i) { return new TestExtension(i); } @@ -37,11 +35,13 @@ public class JavaExtension { system = i; } } - + static class OtherExtension implements Extension { - static final ExtensionKey key = new ExtensionKey(OtherExtension.class) {}; + static final ExtensionKey key = new ExtensionKey(OtherExtension.class) { + }; public final ActorSystemImpl system; + public OtherExtension(ActorSystemImpl i) { system = i; } @@ -51,8 +51,8 @@ public class JavaExtension { @BeforeClass public static void beforeAll() { - Config c = ConfigFactory.parseString("akka.extensions = [ \"akka.actor.JavaExtension$Provider\" ]").withFallback( - AkkaSpec.testConf()); + Config c = ConfigFactory.parseString("akka.extensions = [ \"akka.actor.JavaExtension$TestExtensionId\" ]") + .withFallback(AkkaSpec.testConf()); system = ActorSystem.create("JavaExtension", c); } @@ -64,10 +64,10 @@ public class JavaExtension { @Test public void mustBeAccessible() { - assertSame(system.extension(defaultInstance).system, system); - assertSame(defaultInstance.apply(system).system, system); + assertSame(system.extension(TestExtensionId.instance).system, system); + assertSame(TestExtensionId.instance.apply(system).system, system); } - + @Test public void mustBeAdHoc() { assertSame(OtherExtension.key.apply(system).system, system); diff --git a/akka-docs/general/configuration.rst b/akka-docs/general/configuration.rst index 3cb046d364..8109e7a358 100644 --- a/akka-docs/general/configuration.rst +++ b/akka-docs/general/configuration.rst @@ -189,15 +189,15 @@ before or after using them to construct an actor system: Welcome to Scala version 2.9.1.final (Java HotSpot(TM) 64-Bit Server VM, Java 1.6.0_27). Type in expressions to have them evaluated. Type :help for more information. - + scala> import com.typesafe.config._ import com.typesafe.config._ - + scala> ConfigFactory.parseString("a.b=12") res0: com.typesafe.config.Config = Config(SimpleConfigObject({"a" : {"b" : 12}})) - + scala> res0.root.render - res1: java.lang.String = + res1: java.lang.String = { # String: 1 "a" : { @@ -217,3 +217,12 @@ and parsed by the actor system can be displayed like this: println(system.settings()); // this is a shortcut for system.settings().config().root().render() + +Application specific settings +----------------------------- + +The configuration can also be used for application specific settings. +A good practice is to place those settings in an Extension, as described in: + + * Scala API: :ref:`extending-akka-scala.settings` + * Java API: :ref:`extending-akka-java.settings` diff --git a/akka-docs/java/code/akka/docs/extension/ExtensionDocTestBase.java b/akka-docs/java/code/akka/docs/extension/ExtensionDocTestBase.java index dbc704a2b2..c21f6b7401 100644 --- a/akka-docs/java/code/akka/docs/extension/ExtensionDocTestBase.java +++ b/akka-docs/java/code/akka/docs/extension/ExtensionDocTestBase.java @@ -9,63 +9,63 @@ import java.util.concurrent.atomic.AtomicLong; //#imports -import org.junit.After; -import org.junit.Before; import org.junit.Test; -import static org.junit.Assert.*; public class ExtensionDocTestBase { - //#extension - public static class CountExtensionImpl implements Extension { - //Since this Extension is a shared instance - // per ActorSystem we need to be threadsafe - private final AtomicLong counter = new AtomicLong(0); + //#extension + public static class CountExtensionImpl implements Extension { + //Since this Extension is a shared instance + // per ActorSystem we need to be threadsafe + private final AtomicLong counter = new AtomicLong(0); - //This is the operation this Extension provides - public long increment() { - return counter.incrementAndGet(); - } + //This is the operation this Extension provides + public long increment() { + return counter.incrementAndGet(); } - //#extension + } - //#extensionid - static class CountExtensionId extends AbstractExtensionId { - //This method will be called by Akka - // to instantiate our Extension - public CountExtensionImpl createExtension(ActorSystemImpl i) { - return new CountExtensionImpl(); - } - } + //#extension + //#extensionid + public static class CountExtension extends AbstractExtensionId implements ExtensionIdProvider { //This will be the identifier of our CountExtension - public final static CountExtensionId CountExtension = new CountExtensionId(); - //#extensionid + public final static CountExtension instance = new CountExtension(); - //#extensionid-provider - static class CountExtensionIdProvider implements ExtensionIdProvider { - public CountExtensionId lookup() { - return CountExtension; //The public static final - } - } - //#extensionid-provider - - //#extension-usage-actor - static class MyActor extends UntypedActor { - public void onReceive(Object msg) { - CountExtension.get(getContext().system()).increment(); - } + //The lookup method is required by ExtensionIdProvider, + // so we return ourselves here, this allows us + // to configure our extension to be loaded when + // the ActorSystem starts up + public CountExtension lookup() { + return CountExtension.instance; //The public static final } - //#extension-usage-actor + //This method will be called by Akka + // to instantiate our Extension + public CountExtensionImpl createExtension(ActorSystemImpl system) { + return new CountExtensionImpl(); + } + } - @Test public void demonstrateHowToCreateAndUseAnAkkaExtensionInJava() { + //#extensionid + + //#extension-usage-actor + public static class MyActor extends UntypedActor { + public void onReceive(Object msg) { + CountExtension.instance.get(getContext().system()).increment(); + } + } + + //#extension-usage-actor + + @Test + public void demonstrateHowToCreateAndUseAnAkkaExtensionInJava() { final ActorSystem system = null; try { //#extension-usage - CountExtension.get(system).increment(); + CountExtension.instance.get(system).increment(); //#extension-usage - } catch(Exception e) { + } catch (Exception e) { //do nothing } } diff --git a/akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTest.scala b/akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTest.scala new file mode 100644 index 0000000000..0979c00d4f --- /dev/null +++ b/akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTest.scala @@ -0,0 +1,8 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.docs.extension + +import org.scalatest.junit.JUnitSuite + +class SettingsExtensionDocTest extends SettingsExtensionDocTestBase with JUnitSuite diff --git a/akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTestBase.java b/akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTestBase.java new file mode 100644 index 0000000000..bbe2b2bcdc --- /dev/null +++ b/akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTestBase.java @@ -0,0 +1,85 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.docs.extension; + +//#imports +import akka.actor.Extension; +import akka.actor.AbstractExtensionId; +import akka.actor.ExtensionIdProvider; +import akka.actor.ActorSystem; +import akka.actor.ActorSystemImpl; +import akka.util.Duration; +import com.typesafe.config.Config; +import java.util.concurrent.TimeUnit; + +//#imports + +import akka.actor.UntypedActor; +import org.junit.Test; + +public class SettingsExtensionDocTestBase { + + //#extension + public static class SettingsImpl implements Extension { + + public final String DB_URI; + public final Duration CIRCUIT_BREAKER_TIMEOUT; + + public SettingsImpl(Config config) { + DB_URI = config.getString(config.getString("myapp.db.uri")); + CIRCUIT_BREAKER_TIMEOUT = Duration.create(config.getMilliseconds("myapp.circuit-breaker.timeout"), + TimeUnit.MILLISECONDS); + } + + } + + //#extension + + //#extensionid + public static class Settings extends AbstractExtensionId implements ExtensionIdProvider { + public final static Settings instance = new Settings(); + + public Settings lookup() { + return Settings.instance; + } + + public SettingsImpl createExtension(ActorSystemImpl system) { + return new SettingsImpl(system.settings().config()); + } + } + + //#extensionid + + //#extension-usage-actor + public static class MyActor extends UntypedActor { + Connection connection = connect(Settings.instance.get(getContext().system()).DB_URI, + Settings.instance.get(getContext().system()).CIRCUIT_BREAKER_TIMEOUT); + + //#extension-usage-actor + + public Connection connect(String dbUri, Duration circuitBreakerTimeout) { + return new Connection(); + } + + public void onReceive(Object msg) { + } + + } + + public static class Connection { + } + + @Test + public void demonstrateHowToCreateAndUseAnAkkaExtensionInJava() { + final ActorSystem system = null; + try { + //#extension-usage + String dbUri = Settings.instance.get(system).DB_URI; + //#extension-usage + } catch (Exception e) { + //do nothing + } + } + +} diff --git a/akka-docs/java/extending-akka.rst b/akka-docs/java/extending-akka.rst index edbea859b1..ac60147881 100644 --- a/akka-docs/java/extending-akka.rst +++ b/akka-docs/java/extending-akka.rst @@ -1,7 +1,8 @@ .. _extending-akka-java: -Akka Extensions -=============== +######################## + Akka Extensions (Java) +######################## .. sidebar:: Contents @@ -15,13 +16,13 @@ You can choose to have your Extension loaded on-demand or at ``ActorSystem`` cre Details on how to make that happens are below, in the "Loading from Configuration" section. .. warning:: - + Since an extension is a way to hook into Akka itself, the implementor of the extension needs to ensure the thread safety of his/her extension. Building an Extension ---------------------- +===================== So let's create a sample extension that just lets us count the number of times something has happened. @@ -33,7 +34,7 @@ First, we define what our ``Extension`` should do: Then we need to create an ``ExtensionId`` for our extension so we can grab ahold of it. .. includecode:: code/akka/docs/extension/ExtensionDocTestBase.java - :include: imports,extensionid + :include: imports,extensionid Wicked! Now all we need to do is to actually use it: @@ -48,16 +49,37 @@ Or from inside of an Akka Actor: That's all there is to it! Loading from Configuration --------------------------- +========================== To be able to load extensions from your Akka configuration you must add FQCNs of implementations of either ``ExtensionId`` or ``ExtensionIdProvider`` in the "akka.extensions" section of the config you provide to your ``ActorSystem``. -.. includecode:: code/akka/docs/extension/ExtensionDocTestBase.java - :include: extensionid-provider - Applicability -------------- +============= The sky is the limit! By the way, did you know that Akka's ``Typed Actors``, ``Serialization`` and other features are implemented as Akka Extensions? + +.. _extending-akka-java.settings: + +Application specific settings +----------------------------- + +The :ref:`configuration` can be used for application specific settings. A good practice is to place those settings in an Extension. + +Sample configuration: + +.. includecode:: ../scala/code/akka/docs/extension/SettingsExtensionDocSpec.scala + :include: config + +The ``Extension``: + +.. includecode:: code/akka/docs/extension/SettingsExtensionDocTestBase.java + :include: imports,extension,extensionid + + +Use it: + +.. includecode:: code/akka/docs/extension/SettingsExtensionDocTestBase.java + :include: extension-usage-actor + diff --git a/akka-docs/scala/code/akka/docs/extension/ExtensionDocSpec.scala b/akka-docs/scala/code/akka/docs/extension/ExtensionDocSpec.scala index e9ab9ac8e1..1f4d777ad9 100644 --- a/akka-docs/scala/code/akka/docs/extension/ExtensionDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/extension/ExtensionDocSpec.scala @@ -3,16 +3,13 @@ */ package akka.docs.extension -import org.scalatest.WordSpec -import org.scalatest.matchers.MustMatchers - -//#imports -import akka.actor._ import java.util.concurrent.atomic.AtomicLong - -//#imports +import akka.actor.Actor +import akka.testkit.AkkaSpec //#extension +import akka.actor.Extension + class CountExtensionImpl extends Extension { //Since this Extension is a shared instance // per ActorSystem we need to be threadsafe @@ -24,6 +21,10 @@ class CountExtensionImpl extends Extension { //#extension //#extensionid +import akka.actor.ExtensionId +import akka.actor.ExtensionIdProvider +import akka.actor.ActorSystemImpl + object CountExtension extends ExtensionId[CountExtensionImpl] with ExtensionIdProvider { @@ -39,39 +40,37 @@ object CountExtension } //#extensionid -//#extension-usage-actor -import akka.actor.Actor +object ExtensionDocSpec { + //#extension-usage-actor -class MyActor extends Actor { - def receive = { - case someMessage ⇒ - CountExtension(context.system).increment() - } -} -//#extension-usage-actor - -//#extension-usage-actor-trait -import akka.actor.Actor - -trait Counting { self: Actor ⇒ - def increment() = CountExtension(context.system).increment() -} -class MyCounterActor extends Actor with Counting { - def receive = { - case someMessage ⇒ increment() - } -} -//#extension-usage-actor-trait - -class ExtensionDocSpec extends WordSpec with MustMatchers { - - "demonstrate how to create an extension in Scala" in { - val system: ActorSystem = null - intercept[Exception] { - //#extension-usage - CountExtension(system).increment - //#extension-usage + class MyActor extends Actor { + def receive = { + case someMessage ⇒ + CountExtension(context.system).increment() } } + //#extension-usage-actor + + //#extension-usage-actor-trait + + trait Counting { self: Actor ⇒ + def increment() = CountExtension(context.system).increment() + } + class MyCounterActor extends Actor with Counting { + def receive = { + case someMessage ⇒ increment() + } + } + //#extension-usage-actor-trait +} + +class ExtensionDocSpec extends AkkaSpec { + import ExtensionDocSpec._ + + "demonstrate how to create an extension in Scala" in { + //#extension-usage + CountExtension(system).increment + //#extension-usage + } } diff --git a/akka-docs/scala/code/akka/docs/extension/SettingsExtensionDocSpec.scala b/akka-docs/scala/code/akka/docs/extension/SettingsExtensionDocSpec.scala new file mode 100644 index 0000000000..91e4f0ba1a --- /dev/null +++ b/akka-docs/scala/code/akka/docs/extension/SettingsExtensionDocSpec.scala @@ -0,0 +1,77 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.docs.extension + +//#imports +import akka.actor.Extension +import akka.actor.ExtensionId +import akka.actor.ExtensionIdProvider +import akka.actor.ActorSystemImpl +import akka.util.Duration +import com.typesafe.config.Config +import java.util.concurrent.TimeUnit + +//#imports + +import akka.actor.Actor +import akka.testkit.AkkaSpec + +//#extension +class SettingsImpl(config: Config) extends Extension { + val DbUri: String = config.getString("myapp.db.uri") + val CircuitBreakerTimeout: Duration = Duration(config.getMilliseconds("myapp.circuit-breaker.timeout"), TimeUnit.MILLISECONDS) +} +//#extension + +//#extensionid +object Settings extends ExtensionId[SettingsImpl] with ExtensionIdProvider { + + override def lookup = Settings + + override def createExtension(system: ActorSystemImpl) = new SettingsImpl(system.settings.config) +} +//#extensionid + +object SettingsExtensionDocSpec { + + val config = """ + //#config + myapp { + db { + uri = "mongodb://example1.com:27017,example2.com:27017" + } + circuit-breaker { + timeout = 30 seconds + } + } + //#config + """ + + //#extension-usage-actor + + class MyActor extends Actor { + val connection = connect(Settings(context.system).DbUri, Settings(context.system).CircuitBreakerTimeout) + + //#extension-usage-actor + def receive = { + case someMessage ⇒ + } + + def connect(dbUri: String, circuitBreakerTimeout: Duration) = { + "dummy" + } + } + +} + +class SettingsExtensionDocSpec extends AkkaSpec(SettingsExtensionDocSpec.config) { + + "demonstrate how to create application specific settings extension in Scala" in { + //#extension-usage + val dbUri = Settings(system).DbUri + val circuitBreakerTimeout = Settings(system).CircuitBreakerTimeout + //#extension-usage + } + +} diff --git a/akka-docs/scala/extending-akka.rst b/akka-docs/scala/extending-akka.rst index 6a1655db01..0fe149e0f2 100644 --- a/akka-docs/scala/extending-akka.rst +++ b/akka-docs/scala/extending-akka.rst @@ -1,7 +1,9 @@ .. _extending-akka-scala: -Akka Extensions -=============== +######################### + Akka Extensions (Scala) +######################### + .. sidebar:: Contents @@ -9,19 +11,19 @@ Akka Extensions Building an Extension ---------------------- +===================== So let's create a sample extension that just lets us count the number of times something has happened. First, we define what our ``Extension`` should do: .. includecode:: code/akka/docs/extension/ExtensionDocSpec.scala - :include: imports,extension + :include: extension Then we need to create an ``ExtensionId`` for our extension so we can grab ahold of it. .. includecode:: code/akka/docs/extension/ExtensionDocSpec.scala - :include: imports,extensionid + :include: extensionid Wicked! Now all we need to do is to actually use it: @@ -41,13 +43,37 @@ You can also hide extension behind traits: That's all there is to it! Loading from Configuration --------------------------- +========================== To be able to load extensions from your Akka configuration you must add FQCNs of implementations of either ``ExtensionId`` or ``ExtensionIdProvider`` -in the "akka.extensions" section of the config you provide to your ``ActorSystem``. +in the ``akka.extensions`` section of the config you provide to your ``ActorSystem``. Applicability -------------- +============= The sky is the limit! By the way, did you know that Akka's ``Typed Actors``, ``Serialization`` and other features are implemented as Akka Extensions? + +.. _extending-akka-scala.settings: + +Application specific settings +----------------------------- + +The :ref:`configuration` can be used for application specific settings. A good practice is to place those settings in an Extension. + +Sample configuration: + +.. includecode:: code/akka/docs/extension/SettingsExtensionDocSpec.scala + :include: config + +The ``Extension``: + +.. includecode:: code/akka/docs/extension/SettingsExtensionDocSpec.scala + :include: imports,extension,extensionid + + +Use it: + +.. includecode:: code/akka/docs/extension/SettingsExtensionDocSpec.scala + :include: extension-usage-actor + diff --git a/akka-docs/scala/testing.rst b/akka-docs/scala/testing.rst index 0f80342bc4..fb1513dea4 100644 --- a/akka-docs/scala/testing.rst +++ b/akka-docs/scala/testing.rst @@ -1,8 +1,8 @@ .. _akka-testkit: -##################### -Testing Actor Systems -##################### +############################## +Testing Actor Systems (Scala) +############################## .. toctree:: diff --git a/akka-docs/scala/testkit-example.rst b/akka-docs/scala/testkit-example.rst index 2290cc84af..a87e8fc311 100644 --- a/akka-docs/scala/testkit-example.rst +++ b/akka-docs/scala/testkit-example.rst @@ -1,8 +1,8 @@ .. _testkit-example: -############### -TestKit Example -############### +######################## +TestKit Example (Scala) +######################## Ray Roestenburg's example code from `his blog `_ adapted to work with Akka 1.1. From 8dfe619140387cbf543ccaf1131eaf260e955041 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 20 Jan 2012 12:30:19 +0100 Subject: [PATCH 067/152] #1703 & #1693 - moving daemonicity to one place, and in doing so creating a thread factory in ActorSystem --- .../test/scala/akka/config/ConfigSpec.scala | 5 ++- .../test/scala/akka/routing/RoutingSpec.scala | 5 +-- akka-actor/src/main/resources/reference.conf | 7 ++-- .../main/scala/akka/actor/ActorSystem.scala | 7 ++-- .../akka/dispatch/AbstractDispatcher.scala | 2 +- .../main/scala/akka/dispatch/Dispatcher.scala | 9 ++++- .../scala/akka/dispatch/Dispatchers.scala | 5 +-- .../akka/dispatch/ThreadPoolBuilder.scala | 33 +++++++++++++------ .../akka/docs/actor/TypedActorDocSpec.scala | 2 +- .../docs/dispatcher/DispatcherDocSpec.scala | 2 -- akka-remote/src/main/resources/reference.conf | 5 --- .../scala/akka/remote/RemoteSettings.scala | 1 - .../remote/netty/NettyRemoteSupport.scala | 21 ++++++------ .../scala/akka/remote/RemoteConfigSpec.scala | 1 - 14 files changed, 57 insertions(+), 48 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala index 521e8d4d4a..67c7a51b60 100644 --- a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala @@ -23,6 +23,8 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference) { getString("akka.version") must equal("2.0-SNAPSHOT") settings.ConfigVersion must equal("2.0-SNAPSHOT") + getBoolean("akka.daemonic") must equal(false) + getString("akka.actor.default-dispatcher.type") must equal("Dispatcher") getMilliseconds("akka.actor.default-dispatcher.keep-alive-time") must equal(60 * 1000) getDouble("akka.actor.default-dispatcher.core-pool-size-factor") must equal(3.0) @@ -45,9 +47,6 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference) { getMilliseconds("akka.scheduler.tickDuration") must equal(100) settings.SchedulerTickDuration must equal(100 millis) - - getBoolean("akka.scheduler.daemonic") must equal(true) - settings.SchedulerDaemonicity must equal(true) } } } diff --git a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala index bcf3e6328e..9529854314 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala @@ -31,10 +31,7 @@ object RoutingSpec { """ class TestActor extends Actor { - def receive = { - case _ ⇒ - println("Hello") - } + def receive = { case _ ⇒ } } class Echo extends Actor { diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index e90e4f41bf..999c4286c2 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -33,6 +33,9 @@ akka { # See the Akka Documentation for more info about Extensions extensions = [] + # Toggles whether the threads created by this ActorSystem should be daemons or not + daemonic = off + actor { provider = "akka.actor.LocalActorRefProvider" @@ -155,9 +158,6 @@ akka { # parameters type = "Dispatcher" - # Toggles whether the threads created by this dispatcher should be daemons or not - daemonic = off - # Keep alive time for threads keep-alive-time = 60s @@ -271,6 +271,5 @@ akka { # For more information see: http://www.jboss.org/netty/ tickDuration = 100ms ticksPerWheel = 512 - daemonic = on } } diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 0a646709bc..9fc3946808 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -96,7 +96,7 @@ object ActorSystem { final val SchedulerTickDuration = Duration(getMilliseconds("akka.scheduler.tickDuration"), MILLISECONDS) final val SchedulerTicksPerWheel = getInt("akka.scheduler.ticksPerWheel") - final val SchedulerDaemonicity = getBoolean("akka.scheduler.daemonic") + final val Daemonicity = getBoolean("akka.daemonic") if (ConfigVersion != Version) throw new ConfigurationException("Akka JAR version [" + Version + "] does not match the provided config version [" + ConfigVersion + "]") @@ -275,6 +275,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor import ActorSystem._ final val settings = new Settings(applicationConfig, name) + final val threadFactory = new MonitorableThreadFactory(name, settings.Daemonicity) def logConfiguration(): Unit = log.info(settings.toString) @@ -361,7 +362,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor } } - val dispatchers = new Dispatchers(settings, DefaultDispatcherPrerequisites(eventStream, deadLetterMailbox, scheduler)) + val dispatchers = new Dispatchers(settings, DefaultDispatcherPrerequisites(threadFactory, eventStream, deadLetterMailbox, scheduler)) val dispatcher = dispatchers.defaultGlobalDispatcher def terminationFuture: Future[Unit] = provider.terminationFuture @@ -410,7 +411,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor */ protected def createScheduler(): Scheduler = { val hwt = new HashedWheelTimer(log, - new MonitorableThreadFactory("DefaultScheduler", settings.SchedulerDaemonicity), + threadFactory.copy(threadFactory.name + "-scheduler"), settings.SchedulerTickDuration, settings.SchedulerTicksPerWheel) // note that dispatcher is by-name parameter in DefaultScheduler constructor, diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index ff209c8c00..12c8c89d25 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -342,7 +342,7 @@ abstract class MessageDispatcherConfigurator(val config: Config, val prerequisit //Apply the following options to the config if they are present in the config - ThreadPoolConfigDispatcherBuilder(createDispatcher, ThreadPoolConfig(daemonic = config getBoolean "daemonic")) + ThreadPoolConfigDispatcherBuilder(createDispatcher, ThreadPoolConfig()) .setKeepAliveTime(Duration(config getMilliseconds "keep-alive-time", TimeUnit.MILLISECONDS)) .setAllowCoreThreadTimeout(config getBoolean "allow-core-timeout") .setCorePoolSizeFromFactor(config getInt "core-pool-size-min", config getDouble "core-pool-size-factor", config getInt "core-pool-size-max") diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala index 0e72f01681..906c160dce 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala @@ -32,7 +32,14 @@ class Dispatcher( val shutdownTimeout: Duration) extends MessageDispatcher(_prerequisites) { - protected[akka] val executorServiceFactory = executorServiceFactoryProvider.createExecutorServiceFactory(id) + protected[akka] val executorServiceFactory: ExecutorServiceFactory = + executorServiceFactoryProvider.createExecutorServiceFactory( + id, + prerequisites.threadFactory match { + case m: MonitorableThreadFactory ⇒ m.copy(m.name + "-" + id) + case other ⇒ other + }) + protected[akka] val executorService = new AtomicReference[ExecutorService](new ExecutorServiceDelegate { lazy val executor = executorServiceFactory.createExecutorService }) diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala index 31258c540e..b622c52c74 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala @@ -4,8 +4,6 @@ package akka.dispatch -import java.util.concurrent.TimeUnit -import java.util.concurrent.ConcurrentHashMap import akka.actor.newUuid import akka.util.{ Duration, ReflectiveAccess } import akka.actor.ActorSystem @@ -17,14 +15,17 @@ import com.typesafe.config.ConfigFactory import akka.config.ConfigurationException import akka.event.Logging.Warning import akka.actor.Props +import java.util.concurrent.{ ThreadFactory, TimeUnit, ConcurrentHashMap } trait DispatcherPrerequisites { + def threadFactory: ThreadFactory def eventStream: EventStream def deadLetterMailbox: Mailbox def scheduler: Scheduler } case class DefaultDispatcherPrerequisites( + val threadFactory: ThreadFactory, val eventStream: EventStream, val deadLetterMailbox: Mailbox, val scheduler: Scheduler) extends DispatcherPrerequisites diff --git a/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala index e073e18b66..9601c4dd5e 100644 --- a/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala +++ b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala @@ -16,6 +16,7 @@ object ThreadPoolConfig { val defaultCorePoolSize: Int = 16 val defaultMaxPoolSize: Int = 128 val defaultTimeout: Duration = Duration(60000L, TimeUnit.MILLISECONDS) + val defaultRejectionPolicy: RejectedExecutionHandler = new SaneRejectedExecutionHandler() def scaledPoolSize(floor: Int, multiplier: Double, ceiling: Int): Int = { import scala.math.{ min, max } @@ -54,7 +55,7 @@ trait ExecutorServiceFactory { * Generic way to specify an ExecutorService to a Dispatcher, create it with the given name if desired */ trait ExecutorServiceFactoryProvider { - def createExecutorServiceFactory(name: String): ExecutorServiceFactory + def createExecutorServiceFactory(name: String, threadFactory: ThreadFactory): ExecutorServiceFactory } /** @@ -65,16 +66,24 @@ case class ThreadPoolConfig(allowCorePoolTimeout: Boolean = ThreadPoolConfig.def maxPoolSize: Int = ThreadPoolConfig.defaultMaxPoolSize, threadTimeout: Duration = ThreadPoolConfig.defaultTimeout, queueFactory: ThreadPoolConfig.QueueFactory = ThreadPoolConfig.linkedBlockingQueue(), - daemonic: Boolean = false) + rejectionPolicy: RejectedExecutionHandler = ThreadPoolConfig.defaultRejectionPolicy) extends ExecutorServiceFactoryProvider { class ThreadPoolExecutorServiceFactory(val threadFactory: ThreadFactory) extends ExecutorServiceFactory { def createExecutorService: ExecutorService = { - val service = new ThreadPoolExecutor(corePoolSize, maxPoolSize, threadTimeout.length, threadTimeout.unit, queueFactory(), threadFactory, new SaneRejectedExecutionHandler) + val service = new ThreadPoolExecutor( + corePoolSize, + maxPoolSize, + threadTimeout.length, + threadTimeout.unit, + queueFactory(), + threadFactory, + rejectionPolicy) service.allowCoreThreadTimeOut(allowCorePoolTimeout) service } } - final def createExecutorServiceFactory(name: String): ExecutorServiceFactory = new ThreadPoolExecutorServiceFactory(new MonitorableThreadFactory(name, daemonic)) + final def createExecutorServiceFactory(name: String, threadFactory: ThreadFactory): ExecutorServiceFactory = + new ThreadPoolExecutorServiceFactory(threadFactory) } trait DispatcherBuilder { @@ -143,16 +152,20 @@ case class ThreadPoolConfigDispatcherBuilder(dispatcherFactory: (ThreadPoolConfi def configure(fs: Option[Function[ThreadPoolConfigDispatcherBuilder, ThreadPoolConfigDispatcherBuilder]]*): ThreadPoolConfigDispatcherBuilder = fs.foldLeft(this)((c, f) ⇒ f.map(_(c)).getOrElse(c)) } -class MonitorableThreadFactory(val name: String, val daemonic: Boolean = false) extends ThreadFactory { +object MonitorableThreadFactory { + val doNothing: Thread.UncaughtExceptionHandler = + new Thread.UncaughtExceptionHandler() { def uncaughtException(thread: Thread, cause: Throwable) = () } +} + +case class MonitorableThreadFactory(name: String, + daemonic: Boolean, + exceptionHandler: Thread.UncaughtExceptionHandler = MonitorableThreadFactory.doNothing) + extends ThreadFactory { protected val counter = new AtomicLong - protected val doNothing: Thread.UncaughtExceptionHandler = - new Thread.UncaughtExceptionHandler() { - def uncaughtException(thread: Thread, cause: Throwable) = {} - } def newThread(runnable: Runnable) = { val t = new Thread(runnable, name + counter.incrementAndGet()) - t.setUncaughtExceptionHandler(doNothing) + t.setUncaughtExceptionHandler(exceptionHandler) t.setDaemon(daemonic) t } diff --git a/akka-docs/scala/code/akka/docs/actor/TypedActorDocSpec.scala b/akka-docs/scala/code/akka/docs/actor/TypedActorDocSpec.scala index f292b39a7c..da718b503d 100644 --- a/akka-docs/scala/code/akka/docs/actor/TypedActorDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/actor/TypedActorDocSpec.scala @@ -45,7 +45,7 @@ class SquarerImpl(val name: String) extends Squarer { //#typed-actor-impl-methods } //#typed-actor-impl - +import java.lang.Integer.{ parseInt ⇒ println } //Mr funny man avoids printing to stdout AND keeping docs alright //#typed-actor-supercharge trait Foo { def doFoo(times: Int): Unit = println("doFoo(" + times + ")") diff --git a/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala b/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala index 2747da9f91..d0e0945fe8 100644 --- a/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala @@ -22,8 +22,6 @@ object DispatcherDocSpec { my-dispatcher { # Dispatcher is the name of the event-based dispatcher type = Dispatcher - # Toggles whether the threads created by this dispatcher should be daemons or not - daemonic = off # minimum number of threads to cap factor-based core number to core-pool-size-min = 2 # No of core threads ... ceil(available processors * factor) diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index a89508982c..cac020bf70 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -59,9 +59,6 @@ akka { # Reuse inbound connections for outbound messages use-passive-connections = on - # Whether any Threds created by the remoting should be daemons or not - daemonic = on - # accrual failure detection config failure-detector { @@ -84,13 +81,11 @@ akka { compute-grid-dispatcher { # defaults to same settings as default-dispatcher name = ComputeGridDispatcher - daemonic = on } # The dispatcher used for the system actor "network-event-sender" network-event-sender-dispatcher { type = PinnedDispatcher - daemonic = on } server { diff --git a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala index ad4fd2625e..4a3e869271 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala @@ -16,7 +16,6 @@ class RemoteSettings(val config: Config, val systemName: String) { import config._ val RemoteTransport = getString("akka.remote.transport") - val Daemonic = getBoolean("akka.remote.daemonic") val FailureDetectorThreshold = getInt("akka.remote.failure-detector.threshold") val FailureDetectorMaxSampleSize = getInt("akka.remote.failure-detector.max-sample-size") val ShouldCompressData = getBoolean("akka.remote.use-compression") diff --git a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala index bfb30bc940..600e49290b 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -369,7 +369,7 @@ class NettyRemoteSupport(_system: ActorSystemImpl, val remote: Remote, val addre val serverSettings = remote.remoteSettings.serverSettings val clientSettings = remote.remoteSettings.clientSettings - val threadFactory = new MonitorableThreadFactory("NettyRemoteSupport", remote.remoteSettings.Daemonic) + val threadFactory = _system.threadFactory.copy(_system.threadFactory.name + "-remote") val timer: HashedWheelTimer = new HashedWheelTimer val executor = new OrderedMemoryAwareThreadPoolExecutor( serverSettings.ExecutionPoolSize, @@ -535,23 +535,24 @@ class NettyRemoteServer( Executors.newCachedThreadPool(remoteSupport.threadFactory), Executors.newCachedThreadPool(remoteSupport.threadFactory)) - private val bootstrap = new ServerBootstrap(factory) - private val executionHandler = new ExecutionHandler(remoteSupport.executor) // group of open channels, used for clean-up private val openChannels: ChannelGroup = new DefaultDisposableChannelGroup("akka-remote-server") val pipelineFactory = new RemoteServerPipelineFactory(name, openChannels, executionHandler, loader, remoteSupport) - bootstrap.setPipelineFactory(pipelineFactory) - bootstrap.setOption("backlog", Backlog) - bootstrap.setOption("child.tcpNoDelay", true) - bootstrap.setOption("child.keepAlive", true) - bootstrap.setOption("child.reuseAddress", true) - bootstrap.setOption("child.connectTimeoutMillis", ConnectionTimeout.toMillis) + private val bootstrap: ServerBootstrap = { + val b = new ServerBootstrap(factory) + b.setPipelineFactory(pipelineFactory) + b.setOption("backlog", Backlog) + b.setOption("child.tcpNoDelay", true) + b.setOption("child.keepAlive", true) + b.setOption("child.reuseAddress", true) + b.setOption("child.connectTimeoutMillis", ConnectionTimeout.toMillis) + b + } openChannels.add(bootstrap.bind(new InetSocketAddress(address.transport.ip.get, address.transport.port))) - remoteSupport.notifyListeners(RemoteServerStarted(remoteSupport)) def shutdown() { try { diff --git a/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala index 49502b99ea..03a343f3b1 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala @@ -16,7 +16,6 @@ class RemoteConfigSpec extends AkkaSpec("") { getString("akka.remote.secure-cookie") must equal("") getBoolean("akka.remote.use-passive-connections") must equal(true) getMilliseconds("akka.remote.backoff-timeout") must equal(0) - getBoolean("akka.remote.daemonic") must equal(true) // getMilliseconds("akka.remote.remote-daemon-ack-timeout") must equal(30 * 1000) //akka.remote.server From e6cee135f0f569f723b04ddd90276d1457baf9cc Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 20 Jan 2012 14:17:27 +0100 Subject: [PATCH 068/152] Run callbacks in defined order, and ActorSystem.awaitTermination. See #1704, #1705 --- .../scala/akka/actor/ActorSystemSpec.scala | 51 +++++++++++ .../main/scala/akka/actor/ActorSystem.scala | 88 ++++++++++++++++--- 2 files changed, 127 insertions(+), 12 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala index 3ddb616982..f5d6717627 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala @@ -6,6 +6,11 @@ package akka.actor import akka.testkit._ import org.scalatest.junit.JUnitSuite import com.typesafe.config.ConfigFactory +import java.util.concurrent.ConcurrentLinkedQueue +import java.util.concurrent.CountDownLatch +import akka.dispatch.Await +import akka.util.duration._ +import scala.collection.JavaConverters class JavaExtensionSpec extends JavaExtension with JUnitSuite @@ -17,6 +22,7 @@ object TestExtension extends ExtensionId[TestExtension] with ExtensionIdProvider // Dont't place inside ActorSystemSpec object, since it will not be garbage collected and reference to system remains class TestExtension(val system: ActorSystemImpl) extends Extension +@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExtension$"]""") { "An ActorSystem" must { @@ -27,6 +33,51 @@ class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExt system.hasExtension(TestExtension) must be(true) } + "run termination callbacks in order" in { + import scala.collection.JavaConverters._ + + val system2 = ActorSystem("TerminationCallbacks", AkkaSpec.testConf) + val result = new ConcurrentLinkedQueue[Int] + val count = 10 + val latch = TestLatch(count) + + for (i ← 1 to count) { + system2.registerOnTermination { + (i % 3).millis.dilated.sleep() + result add i + latch.countDown() + } + } + + system2.shutdown() + Await.ready(latch, 5 seconds) + + val expected = (for (i ← 1 to count) yield i).reverse + result.asScala.toSeq must be(expected) + + } + + "awaitTtermination after termination callbacks" in { + import scala.collection.JavaConverters._ + + val system2 = ActorSystem("AwaitTermination", AkkaSpec.testConf) + @volatile + var callbackWasRun = false + + system2.registerOnTermination { + 50.millis.dilated.sleep() + callbackWasRun = true + } + + system2.scheduler.scheduleOnce(200.millis.dilated) { + system2.shutdown() + } + + system2.awaitTermination() + callbackWasRun must be(true) + + } + } } \ No newline at end of file diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 0a646709bc..b6d9ec5a90 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -25,6 +25,11 @@ import java.util.concurrent.{ CountDownLatch, Executors, ConcurrentHashMap } import scala.annotation.tailrec import org.jboss.netty.akka.util.internal.ConcurrentIdentityHashMap import java.io.Closeable +import java.util.concurrent.ConcurrentLinkedQueue +import akka.dispatch.Await.Awaitable +import akka.dispatch.Await.CanAwait +import java.util.concurrent.TimeUnit +import java.util.concurrent.TimeoutException object ActorSystem { @@ -224,22 +229,43 @@ abstract class ActorSystem extends ActorRefFactory { def dispatcher: MessageDispatcher /** - * Register a block of code to run after all actors in this actor system have - * been stopped. Multiple code blocks may be registered by calling this method multiple times; there is no - * guarantee that they will be executed in a particular order. + * Register a block of code (callback) to run after all actors in this actor system have + * been stopped. Multiple code blocks may be registered by calling this method multiple times. + * The callbacks will be run sequentilly in reverse order of registration, i.e. + * last registration is run first. + * + * Callbacks registered after that the shutdown process has started will likely not be run. */ - def registerOnTermination[T](code: ⇒ T) + def registerOnTermination[T](code: ⇒ T): Unit /** - * Register a block of code to run after all actors in this actor system have - * been stopped. Multiple code blocks may be registered by calling this method multiple times; there is no - * guarantee that they will be executed in a particular order (Java API). + * Register a block of code (callback) to run after all actors in this actor system have + * been stopped. Multiple code blocks may be registered by calling this method multiple times. + * The callbacks will be run sequentilly in reverse order of registration, i.e. + * last registration is run first. + * + * Callbacks registered after that the shutdown process has started will likely not be run. + * + * Java API */ - def registerOnTermination(code: Runnable) + def registerOnTermination(code: Runnable): Unit + + /** + * Block current thread until the system has been shutdown, or the specified + * timeout has elapsed. This will block until after all on termination + * callbacks have been run. + */ + def awaitTermination(timeout: Duration = Long.MaxValue.nanos): Unit + + /** + * Block current thread until the system has been shutdown. This will + * block until after all on termination callbacks have been run. + */ + def awaitTermination(): Unit /** * Stop this actor system. This will stop the guardian actor, which in turn - * will recursively stop all its child actors, then the system guardian + * will recursively stop all its child actTAors, then the system guardian * (below which the logging actors reside) and the execute all registered * termination handlers (see [[ActorSystem.registerOnTermination]]). */ @@ -378,8 +404,8 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor provider.init(this) _log = new BusLogging(eventStream, "ActorSystem(" + lookupRoot.path.address + ")", this.getClass) deadLetters.init(dispatcher, lookupRoot.path / "deadLetters") - // this starts the reaper actor and the user-configured logging subscribers, which are also actors registerOnTermination(stopScheduler()) + // this starts the reaper actor and the user-configured logging subscribers, which are also actors _locker = new Locker(scheduler, ReaperInterval, lookupRoot.path / "locker", deathWatch) loadExtensions() if (LogConfigOnStart) logConfiguration() @@ -392,8 +418,15 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor def start() = _start - def registerOnTermination[T](code: ⇒ T) { terminationFuture onComplete (_ ⇒ code) } - def registerOnTermination(code: Runnable) { terminationFuture onComplete (_ ⇒ code.run) } + private lazy val terminationCallbacks = { + val callbacks = new TerminationCallbacks + terminationFuture onComplete (_ ⇒ callbacks.run) + callbacks + } + def registerOnTermination[T](code: ⇒ T) { registerOnTermination(new Runnable { def run = code }) } + def registerOnTermination(code: Runnable) { terminationCallbacks.add(code) } + def awaitTermination(timeout: Duration) { Await.ready(terminationCallbacks, timeout) } + def awaitTermination() = awaitTermination(Long.MaxValue.nanos) def shutdown() { stop(guardian) @@ -494,4 +527,35 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor } override def toString = lookupRoot.path.root.address.toString + + class TerminationCallbacks extends Runnable with Awaitable[Unit] { + import scala.collection.JavaConverters._ + private val callbacks = new ConcurrentLinkedQueue[Runnable] + private val latch = new CountDownLatch(1) + + def add(callback: Runnable) { + callbacks add callback + } + + def run() { + for (c ← callbacks.asScala.toSeq.reverse) { + try { + c.run() + } catch { + case e: Exception ⇒ + log.error(e, "Failed to run termination callback, due to [{}]", e.getMessage) + } + } + latch.countDown() + } + + def ready(atMost: Duration)(implicit permit: CanAwait) = { + val opened = latch.await(atMost.toNanos, TimeUnit.NANOSECONDS) + if (!opened) throw new TimeoutException("Await termination timed out after [%s]" format (atMost.toString)) + this + } + + def result(atMost: Duration)(implicit permit: CanAwait): Unit = ready(atMost) + + } } From 0bf87613d2b4c79432a921fd3ba1efbfbd4604fe Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 20 Jan 2012 14:24:20 +0100 Subject: [PATCH 069/152] Timeout in test --- .../src/test/scala/akka/actor/ActorSystemSpec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala index f5d6717627..25a4804c9c 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala @@ -73,7 +73,7 @@ class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExt system2.shutdown() } - system2.awaitTermination() + system2.awaitTermination(5 seconds) callbackWasRun must be(true) } From cb86591656629f1d69d5c2a558eab5420dd49042 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 20 Jan 2012 14:27:00 +0100 Subject: [PATCH 070/152] #1657 - trying to get random port for port 0 --- .../src/main/scala/akka/remote/RemoteSettings.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala index ad4fd2625e..94f5c87e93 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala @@ -66,7 +66,13 @@ class RemoteSettings(val config: Config, val systemName: String) { case "" ⇒ InetAddress.getLocalHost.getHostAddress case value ⇒ value } - val Port = getInt("akka.remote.server.port") + val Port = getInt("akka.remote.server.port") match { + case 0 => try { + val s = new java.net.ServerSocket(0) + try s.getLocalPort finally s.close() + } catch { case e => throw new ConfigurationException("Unable to obtain random port", e) } + case other => other + } val ConnectionTimeout = Duration(getMilliseconds("akka.remote.server.connection-timeout"), MILLISECONDS) val Backlog = getInt("akka.remote.server.backlog") From daff1fd2a0182bb3809b33a3910bfddf71a0d4e7 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Fri, 20 Jan 2012 15:17:19 +0100 Subject: [PATCH 071/152] Improvements based on feedback --- .../main/scala/akka/actor/ActorSystem.scala | 20 ++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index b6d9ec5a90..b607bc6a7e 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -255,7 +255,7 @@ abstract class ActorSystem extends ActorRefFactory { * timeout has elapsed. This will block until after all on termination * callbacks have been run. */ - def awaitTermination(timeout: Duration = Long.MaxValue.nanos): Unit + def awaitTermination(timeout: Duration): Unit /** * Block current thread until the system has been shutdown. This will @@ -265,7 +265,7 @@ abstract class ActorSystem extends ActorRefFactory { /** * Stop this actor system. This will stop the guardian actor, which in turn - * will recursively stop all its child actTAors, then the system guardian + * will recursively stop all its child actors, then the system guardian * (below which the logging actors reside) and the execute all registered * termination handlers (see [[ActorSystem.registerOnTermination]]). */ @@ -426,7 +426,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor def registerOnTermination[T](code: ⇒ T) { registerOnTermination(new Runnable { def run = code }) } def registerOnTermination(code: Runnable) { terminationCallbacks.add(code) } def awaitTermination(timeout: Duration) { Await.ready(terminationCallbacks, timeout) } - def awaitTermination() = awaitTermination(Long.MaxValue.nanos) + def awaitTermination() = awaitTermination(Duration.Inf) def shutdown() { stop(guardian) @@ -537,21 +537,27 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor callbacks add callback } - def run() { + def run(): Unit = try { for (c ← callbacks.asScala.toSeq.reverse) { try { c.run() } catch { - case e: Exception ⇒ + case e ⇒ + // TODO catching all and continue isn't good for OOME, ticket #1310 log.error(e, "Failed to run termination callback, due to [{}]", e.getMessage) } } + } finally { latch.countDown() } def ready(atMost: Duration)(implicit permit: CanAwait) = { - val opened = latch.await(atMost.toNanos, TimeUnit.NANOSECONDS) - if (!opened) throw new TimeoutException("Await termination timed out after [%s]" format (atMost.toString)) + if (atMost == Duration.Inf) { + latch.await() + } else { + val opened = latch.await(atMost.toNanos, TimeUnit.NANOSECONDS) + if (!opened) throw new TimeoutException("Await termination timed out after [%s]" format (atMost.toString)) + } this } From 47c2b3000a2d28b23201e20cf079bd0fa3fffb0d Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 20 Jan 2012 15:36:56 +0100 Subject: [PATCH 072/152] Moving connection-timeout to client since it belongs there, also adding more docs to the remote reference config --- akka-remote/src/main/resources/reference.conf | 16 +++++++++++----- .../scala/akka/remote/RemoteSettings.scala | 8 ++++---- .../akka/remote/netty/NettyRemoteSupport.scala | 18 ++++++++++-------- .../scala/akka/remote/RemoteConfigSpec.scala | 17 ++--------------- 4 files changed, 27 insertions(+), 32 deletions(-) diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index a89508982c..de143ddbb5 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -105,9 +105,6 @@ akka { # Increase this if you want to be able to send messages with large payloads message-frame-size = 1 MiB - # Timeout duration - connection-timeout = 120s - # Should the remote server require that it peers share the same secure-cookie # (defined in the 'remote' section)? require-cookie = off @@ -133,11 +130,20 @@ akka { } client { + # Time before an attempted connection is considered failed + connection-timeout = 10s + + #Time between each reconnection attempt reconnect-delay = 5s - read-timeout = 3600s - message-frame-size = 1 MiB + # Maximum time window that a client should try to reconnect for reconnection-time-window = 600s + + #Period of time of connection inactivity to be tolerated before hanging up + read-timeout = 3600s + + #Max size per message + message-frame-size = 1 MiB } } diff --git a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala index 94f5c87e93..59e4de5702 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala @@ -39,6 +39,7 @@ class RemoteSettings(val config: Config, val systemName: String) { case cookie ⇒ Some(cookie) } + val ConnectionTimeout = Duration(getMilliseconds("akka.remote.client.connection-timeout"), MILLISECONDS) val ReconnectionTimeWindow = Duration(getMilliseconds("akka.remote.client.reconnection-time-window"), MILLISECONDS) val ReadTimeout = Duration(getMilliseconds("akka.remote.client.read-timeout"), MILLISECONDS) val ReconnectDelay = Duration(getMilliseconds("akka.remote.client.reconnect-delay"), MILLISECONDS) @@ -67,13 +68,12 @@ class RemoteSettings(val config: Config, val systemName: String) { case value ⇒ value } val Port = getInt("akka.remote.server.port") match { - case 0 => try { + case 0 ⇒ try { val s = new java.net.ServerSocket(0) try s.getLocalPort finally s.close() - } catch { case e => throw new ConfigurationException("Unable to obtain random port", e) } - case other => other + } catch { case e ⇒ throw new ConfigurationException("Unable to obtain random port", e) } + case other ⇒ other } - val ConnectionTimeout = Duration(getMilliseconds("akka.remote.server.connection-timeout"), MILLISECONDS) val Backlog = getInt("akka.remote.server.backlog") diff --git a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala index f7d6b1d8b3..a8d25884b8 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -184,6 +184,7 @@ class ActiveRemoteClient private[akka] ( bootstrap.setPipelineFactory(new ActiveRemoteClientPipelineFactory(name, bootstrap, executionHandler, remoteAddress, this)) bootstrap.setOption("tcpNoDelay", true) bootstrap.setOption("keepAlive", true) + bootstrap.setOption("connectTimeoutMillis", ConnectionTimeout.toMillis) log.debug("Starting remote client connection to [{}]", remoteAddress) @@ -548,20 +549,21 @@ class NettyRemoteServer( Executors.newCachedThreadPool(remoteSupport.threadFactory), Executors.newCachedThreadPool(remoteSupport.threadFactory)) - private val bootstrap = new ServerBootstrap(factory) - private val executionHandler = new ExecutionHandler(remoteSupport.executor) // group of open channels, used for clean-up private val openChannels: ChannelGroup = new DefaultDisposableChannelGroup("akka-remote-server") val pipelineFactory = new RemoteServerPipelineFactory(name, openChannels, executionHandler, loader, remoteSupport) - bootstrap.setPipelineFactory(pipelineFactory) - bootstrap.setOption("backlog", Backlog) - bootstrap.setOption("child.tcpNoDelay", true) - bootstrap.setOption("child.keepAlive", true) - bootstrap.setOption("child.reuseAddress", true) - bootstrap.setOption("child.connectTimeoutMillis", ConnectionTimeout.toMillis) + private val bootstrap: ServerBootstrap = { + val b = new ServerBootstrap(factory) + b.setPipelineFactory(pipelineFactory) + b.setOption("backlog", Backlog) + b.setOption("child.tcpNoDelay", true) + b.setOption("child.keepAlive", true) + b.setOption("child.reuseAddress", true) + b + } openChannels.add(bootstrap.bind(new InetSocketAddress(address.transport.ip.get, address.transport.port))) remoteSupport.notifyListeners(RemoteServerStarted(remoteSupport)) diff --git a/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala index 49502b99ea..9f498e6273 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala @@ -22,7 +22,7 @@ class RemoteConfigSpec extends AkkaSpec("") { //akka.remote.server getInt("akka.remote.server.port") must equal(2552) getBytes("akka.remote.server.message-frame-size") must equal(1048576L) - getMilliseconds("akka.remote.server.connection-timeout") must equal(120 * 1000) + getBoolean("akka.remote.server.require-cookie") must equal(false) getBoolean("akka.remote.server.untrusted-mode") must equal(false) getInt("akka.remote.server.backlog") must equal(4096) @@ -38,24 +38,11 @@ class RemoteConfigSpec extends AkkaSpec("") { getMilliseconds("akka.remote.client.reconnect-delay") must equal(5 * 1000) getMilliseconds("akka.remote.client.read-timeout") must equal(3600 * 1000) getMilliseconds("akka.remote.client.reconnection-time-window") must equal(600 * 1000) + getMilliseconds("akka.remote.client.connection-timeout") must equal(10000) // TODO cluster config will go into akka-cluster/reference.conf when we enable that module //akka.cluster getStringList("akka.cluster.seed-nodes") must equal(new java.util.ArrayList[String]) - - // getMilliseconds("akka.cluster.max-time-to-wait-until-connected") must equal(30 * 1000) - // getMilliseconds("akka.cluster.session-timeout") must equal(60 * 1000) - // getMilliseconds("akka.cluster.connection-timeout") must equal(60 * 1000) - // getBoolean("akka.cluster.include-ref-node-in-replica-set") must equal(true) - // getString("akka.cluster.log-directory") must equal("_akka_cluster") - - // //akka.cluster.replication - // getString("akka.cluster.replication.digest-type") must equal("MAC") - // getString("akka.cluster.replication.password") must equal("secret") - // getInt("akka.cluster.replication.ensemble-size") must equal(3) - // getInt("akka.cluster.replication.quorum-size") must equal(2) - // getInt("akka.cluster.replication.snapshot-frequency") must equal(1000) - // getMilliseconds("akka.cluster.replication.timeout") must equal(30 * 1000) } } } From 020c6b61dac47168e961943b46abff80fb875326 Mon Sep 17 00:00:00 2001 From: Roland Date: Thu, 19 Jan 2012 15:13:10 +0100 Subject: [PATCH 073/152] move all Ask stuff to akka.pattern MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - remove ?(msg, timeout), should always use ?(msg)(timeout) because of Scala’s only Martin-acknowledged design flaw of being able to pass tuples into single-arg methods without adding another pair of parens - put a provider into all actor refs, because they all are created by and associated with one - treat all terminated refs equally: tell(msg) and return broken promise --- .../scala/akka/actor/ActorLookupSpec.scala | 10 ++-- .../test/scala/akka/actor/ActorRefSpec.scala | 11 ++-- .../scala/akka/actor/ActorTimeoutSpec.scala | 2 +- .../scala/akka/actor/SupervisorSpec.scala | 10 ++-- .../test/scala/akka/dispatch/FutureSpec.scala | 12 ++-- .../src/test/scala/akka/pattern/AskSpec.scala | 35 ++++++++++++ .../src/main/scala/akka/actor/ActorRef.scala | 57 +++++++++++++------ .../scala/akka/actor/ActorRefProvider.scala | 6 +- .../main/scala/akka/actor/ActorSystem.scala | 4 +- .../src/main/scala/akka/actor/Locker.scala | 7 ++- .../main/scala/akka/actor/TypedActor.scala | 6 +- .../src/main/scala/akka/event/Logging.scala | 1 + .../main/scala/akka/pattern/AskSupport.scala | 20 ++----- .../src/main/scala/akka/pattern/package.scala | 17 +++--- .../src/main/scala/akka/util/Duration.scala | 1 - .../src/main/scala/akka/agent/Agent.scala | 4 +- .../code/akka/docs/actor/ActorDocSpec.scala | 2 +- .../src/main/scala/akka/remote/Remote.scala | 2 +- .../akka/remote/RemoteActorRefProvider.scala | 2 +- 19 files changed, 134 insertions(+), 75 deletions(-) create mode 100644 akka-actor-tests/src/test/scala/akka/pattern/AskSpec.scala diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala index 135a7efe0a..f8bb976b3c 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorLookupSpec.scala @@ -40,11 +40,13 @@ class ActorLookupSpec extends AkkaSpec with DefaultTimeout { val c2 = system.actorOf(p, "c2") val c21 = Await.result((c2 ? Create("c21")).mapTo[ActorRef], timeout.duration) - val user = system.asInstanceOf[ActorSystemImpl].guardian - val syst = system.asInstanceOf[ActorSystemImpl].systemGuardian - val root = system.asInstanceOf[ActorSystemImpl].lookupRoot + val sysImpl = system.asInstanceOf[ActorSystemImpl] - def empty(path: String) = new EmptyLocalActorRef(system.eventStream, system.dispatcher, path match { + val user = sysImpl.guardian + val syst = sysImpl.systemGuardian + val root = sysImpl.lookupRoot + + def empty(path: String) = new EmptyLocalActorRef(system.eventStream, sysImpl.provider, system.dispatcher, path match { case RelativeActorPath(elems) ⇒ system.actorFor("/").path / elems }) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala index 8dbd1c9a8c..a46b0a3a42 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala @@ -288,7 +288,8 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout { val baos = new ByteArrayOutputStream(8192 * 32) val out = new ObjectOutputStream(baos) - val addr = system.asInstanceOf[ActorSystemImpl].provider.rootPath.address + val sysImpl = system.asInstanceOf[ActorSystemImpl] + val addr = sysImpl.provider.rootPath.address val serialized = SerializedActorRef(addr + "/non-existing") out.writeObject(serialized) @@ -296,9 +297,9 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout { out.flush out.close - Serialization.currentSystem.withValue(system.asInstanceOf[ActorSystemImpl]) { + Serialization.currentSystem.withValue(sysImpl) { val in = new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray)) - in.readObject must be === new EmptyLocalActorRef(system.eventStream, system.dispatcher, system.actorFor("/").path / "non-existing") + in.readObject must be === new EmptyLocalActorRef(system.eventStream, sysImpl.provider, system.dispatcher, system.actorFor("/").path / "non-existing") } } @@ -359,8 +360,8 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout { } })) - val ffive = (ref ? (5, timeout)).mapTo[String] - val fnull = (ref ? (null, timeout)).mapTo[String] + val ffive = (ref.ask(5)(timeout)).mapTo[String] + val fnull = (ref.ask(null)(timeout)).mapTo[String] ref ! PoisonPill Await.result(ffive, timeout.duration) must be("five") diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala index d8977aa25d..dac38a7481 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala @@ -45,7 +45,7 @@ class ActorTimeoutSpec extends AkkaSpec with BeforeAndAfterAll with DefaultTimeo "use explicitly supplied timeout" in { within(testTimeout - 100.millis, testTimeout + 300.millis) { val echo = system.actorOf(Props.empty) - val f = echo.?("hallo", testTimeout) + val f = echo.?("hallo")(testTimeout) try { intercept[AskTimeoutException] { Await.result(f, testTimeout + 300.millis) } } finally { system.stop(echo) } diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala index 73ff90c3c7..5995c47d7b 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala @@ -129,12 +129,12 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende } def ping(pingPongActor: ActorRef) = { - Await.result(pingPongActor.?(Ping, TimeoutMillis), TimeoutMillis millis) must be === PongMessage + Await.result(pingPongActor.?(Ping)(TimeoutMillis), TimeoutMillis millis) must be === PongMessage expectMsg(Timeout, PingMessage) } def kill(pingPongActor: ActorRef) = { - val result = (pingPongActor ? (DieReply, TimeoutMillis)) + val result = (pingPongActor.?(DieReply)(TimeoutMillis)) expectMsg(Timeout, ExceptionMessage) intercept[RuntimeException] { Await.result(result, TimeoutMillis millis) } } @@ -152,7 +152,7 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende "not restart temporary actor" in { val (temporaryActor, _) = temporaryActorAllForOne - intercept[RuntimeException] { Await.result(temporaryActor.?(DieReply, TimeoutMillis), TimeoutMillis millis) } + intercept[RuntimeException] { Await.result(temporaryActor.?(DieReply)(TimeoutMillis), TimeoutMillis millis) } expectNoMsg(1 second) } @@ -298,11 +298,11 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende filterEvents(EventFilter[RuntimeException]("Expected", occurrences = 1), EventFilter[IllegalStateException]("error while creating actor", occurrences = 1)) { intercept[RuntimeException] { - Await.result(dyingActor.?(DieReply, TimeoutMillis), TimeoutMillis millis) + Await.result(dyingActor.?(DieReply)(TimeoutMillis), TimeoutMillis millis) } } - Await.result(dyingActor.?(Ping, TimeoutMillis), TimeoutMillis millis) must be === PongMessage + Await.result(dyingActor.?(Ping)(TimeoutMillis), TimeoutMillis millis) must be === PongMessage inits.get must be(3) diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala index 2ee7c4e6c7..44509b270f 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala @@ -324,7 +324,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa })) } val timeout = 10000 - def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) ⇒ actor.?((idx, idx * 200), timeout).mapTo[Int] } + def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) ⇒ actor.?((idx, idx * 200))(timeout).mapTo[Int] } Await.result(Future.fold(futures)(0)(_ + _), timeout millis) must be(45) } @@ -334,7 +334,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa def receive = { case (add: Int, wait: Int) ⇒ Thread.sleep(wait); sender.tell(add) } })) } - def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) ⇒ actor.?((idx, idx * 200), 10000).mapTo[Int] } + def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) ⇒ actor.?((idx, idx * 200))(10000).mapTo[Int] } Await.result(futures.foldLeft(Future(0))((fr, fa) ⇒ for (r ← fr; a ← fa) yield (r + a)), timeout.duration) must be(45) } @@ -351,7 +351,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa })) } val timeout = 10000 - def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) ⇒ actor.?((idx, idx * 100), timeout).mapTo[Int] } + def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) ⇒ actor.?((idx, idx * 100))(timeout).mapTo[Int] } intercept[Throwable] { Await.result(Future.fold(futures)(0)(_ + _), timeout millis) }.getMessage must be("shouldFoldResultsWithException: expected") } } @@ -383,7 +383,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa })) } val timeout = 10000 - def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) ⇒ actor.?((idx, idx * 200), timeout).mapTo[Int] } + def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) ⇒ actor.?((idx, idx * 200))(timeout).mapTo[Int] } assert(Await.result(Future.reduce(futures)(_ + _), timeout millis) === 45) } @@ -400,7 +400,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa })) } val timeout = 10000 - def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) ⇒ actor.?((idx, idx * 100), timeout).mapTo[Int] } + def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) ⇒ actor.?((idx, idx * 100))(timeout).mapTo[Int] } intercept[Throwable] { Await.result(Future.reduce(futures)(_ + _), timeout millis) }.getMessage must be === "shouldFoldResultsWithException: expected" } } @@ -441,7 +441,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa "shouldHandleThrowables" in { class ThrowableTest(m: String) extends Throwable(m) - filterException[ThrowableTest] { + EventFilter[ThrowableTest](occurrences = 4) intercept { val f1 = Future[Any] { throw new ThrowableTest("test") } intercept[ThrowableTest] { Await.result(f1, timeout.duration) } diff --git a/akka-actor-tests/src/test/scala/akka/pattern/AskSpec.scala b/akka-actor-tests/src/test/scala/akka/pattern/AskSpec.scala new file mode 100644 index 0000000000..44cdd91eba --- /dev/null +++ b/akka-actor-tests/src/test/scala/akka/pattern/AskSpec.scala @@ -0,0 +1,35 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.pattern + +import akka.testkit.AkkaSpec + +class AskSpec extends AkkaSpec { + + "The “ask” pattern" must { + + "return broken promises on DeadLetters" in { + val dead = system.actorFor("/system/deadLetters") + val f = dead ask 42 + f.isCompleted must be(true) + f.value.get match { + case Left(_: AskTimeoutException) ⇒ + case v ⇒ fail(v + " was not Left(AskTimeoutException)") + } + } + + "return broken promises on EmptyLocalActorRefs" in { + val empty = system.actorFor("unknown") + implicit val timeout = system.settings.ActorTimeout + val f = empty ? 3.14 + f.isCompleted must be(true) + f.value.get match { + case Left(_: AskTimeoutException) ⇒ + case v ⇒ fail(v + " was not Left(AskTimeoutException)") + } + } + + } + +} \ No newline at end of file diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 5d07a662b3..cbb21bff78 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -167,13 +167,6 @@ trait LocalRef extends ActorRefScope { final def isLocal = true } -/** - * Trait for matching on ActorRefs which have access to a provider; this is used in akka.pattern.ask. - */ -trait ActorRefWithProvider { this: InternalActorRef ⇒ - def provider: ActorRefProvider -} - /** * Internal trait for assembling all the functionality needed internally on * ActorRefs. NOTE THAT THIS IS NOT A STABLE EXTERNAL INTERFACE! @@ -181,12 +174,25 @@ trait ActorRefWithProvider { this: InternalActorRef ⇒ * DO NOT USE THIS UNLESS INTERNALLY WITHIN AKKA! */ private[akka] abstract class InternalActorRef extends ActorRef with ScalaActorRef { this: ActorRefScope ⇒ + /* + * Actor life-cycle management, invoked only internally (in response to user requests via ActorContext). + */ def resume(): Unit def suspend(): Unit def restart(cause: Throwable): Unit def stop(): Unit def sendSystemMessage(message: SystemMessage): Unit + + /** + * Get a reference to the actor ref provider which created this ref. + */ + def provider: ActorRefProvider + + /** + * Obtain parent of this ref; used by getChild for ".." paths. + */ def getParent: InternalActorRef + /** * Obtain ActorRef by possibly traversing the actor tree or looking it up at * some provider-specific location. This method shall return the end result, @@ -196,6 +202,7 @@ private[akka] abstract class InternalActorRef extends ActorRef with ScalaActorRe * exist, return Nobody. */ def getChild(name: Iterator[String]): InternalActorRef + /** * Scope: if this ref points to an actor which resides within the same JVM, * i.e. whose mailbox is directly reachable etc. @@ -203,8 +210,12 @@ private[akka] abstract class InternalActorRef extends ActorRef with ScalaActorRe def isLocal: Boolean } +/** + * This is an internal look-up failure token, not useful for anything else. + */ private[akka] case object Nobody extends MinimalActorRef { val path = new RootActorPath(new LocalAddress("all-systems"), "/Nobody") + def provider = throw new UnsupportedOperationException("Nobody does not provide") } /** @@ -218,7 +229,7 @@ private[akka] class LocalActorRef private[akka] ( val systemService: Boolean = false, _receiveTimeout: Option[Duration] = None, _hotswap: Stack[PartialFunction[Any, Unit]] = Props.noHotSwap) - extends InternalActorRef with LocalRef with ActorRefWithProvider { + extends InternalActorRef with LocalRef { /* * actorCell.start() publishes actorCell & this to the dispatcher, which @@ -371,8 +382,9 @@ trait MinimalActorRef extends InternalActorRef with LocalRef { } object MinimalActorRef { - def apply(_path: ActorPath)(receive: PartialFunction[Any, Unit]): ActorRef = new MinimalActorRef { + def apply(_path: ActorPath, _provider: ActorRefProvider)(receive: PartialFunction[Any, Unit]): ActorRef = new MinimalActorRef { def path = _path + def provider = _provider override def !(message: Any)(implicit sender: ActorRef = null): Unit = if (receive.isDefinedAt(message)) receive(message) } @@ -393,8 +405,6 @@ trait DeadLetterActorRefLike extends MinimalActorRef { def eventStream: EventStream - @volatile - private var brokenPromise: Future[Any] = _ @volatile private var _path: ActorPath = _ def path: ActorPath = { @@ -402,9 +412,13 @@ trait DeadLetterActorRefLike extends MinimalActorRef { _path } - private[akka] def init(dispatcher: MessageDispatcher, path: ActorPath) { + @volatile + private var _provider: ActorRefProvider = _ + def provider = _provider + + private[akka] def init(provider: ActorRefProvider, path: ActorPath) { _path = path - brokenPromise = Promise.failed(new ActorKilledException("In DeadLetterActorRef - promises are always broken."))(dispatcher) + _provider = provider } override def isTerminated(): Boolean = true @@ -426,16 +440,25 @@ class DeadLetterActorRef(val eventStream: EventStream) extends DeadLetterActorRe * This special dead letter reference has a name: it is that which is returned * by a local look-up which is unsuccessful. */ -class EmptyLocalActorRef(val eventStream: EventStream, _dispatcher: MessageDispatcher, _path: ActorPath) - extends DeadLetterActorRefLike { - init(_dispatcher, _path) +class EmptyLocalActorRef( + val eventStream: EventStream, + _provider: ActorRefProvider, + _dispatcher: MessageDispatcher, + _path: ActorPath) extends DeadLetterActorRefLike { + + init(_provider, _path) + override def !(message: Any)(implicit sender: ActorRef = null): Unit = message match { case d: DeadLetter ⇒ // do NOT form endless loops case _ ⇒ eventStream.publish(DeadLetter(message, sender, this)) } } -class VirtualPathContainer(val path: ActorPath, override val getParent: InternalActorRef, val log: LoggingAdapter) extends MinimalActorRef { +class VirtualPathContainer( + val provider: ActorRefProvider, + val path: ActorPath, + override val getParent: InternalActorRef, + val log: LoggingAdapter) extends MinimalActorRef { private val children = new ConcurrentHashMap[String, InternalActorRef] diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 86673d259b..767309d6e0 100755 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -324,6 +324,8 @@ class LocalActorRefProvider( val path = rootPath / "bubble-walker" + def provider: ActorRefProvider = LocalActorRefProvider.this + override def stop() = stopped switchOn { terminationFuture.complete(causeOfTermination.toLeft(())) } @@ -440,7 +442,7 @@ class LocalActorRefProvider( lazy val systemGuardian: InternalActorRef = actorOf(system, guardianProps.withCreator(new SystemGuardian), rootGuardian, rootPath / "system", true, None) - lazy val tempContainer = new VirtualPathContainer(tempNode, rootGuardian, log) + lazy val tempContainer = new VirtualPathContainer(system.provider, tempNode, rootGuardian, log) def registerTempActor(actorRef: InternalActorRef, path: ActorPath): Unit = { assert(path.parent eq tempNode, "cannot registerTempActor() with anything not obtained from tempPath()") @@ -489,7 +491,7 @@ class LocalActorRefProvider( } else ref.getChild(path.iterator) match { case Nobody ⇒ log.debug("look-up of path sequence '{}' failed", path) - new EmptyLocalActorRef(eventStream, dispatcher, ref.path / path) + new EmptyLocalActorRef(eventStream, system.provider, dispatcher, ref.path / path) case x ⇒ x } diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 04084e2c2c..8ff4ebe373 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -377,10 +377,10 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor // the provider is expected to start default loggers, LocalActorRefProvider does this provider.init(this) _log = new BusLogging(eventStream, "ActorSystem(" + lookupRoot.path.address + ")", this.getClass) - deadLetters.init(dispatcher, lookupRoot.path / "deadLetters") + deadLetters.init(provider, lookupRoot.path / "deadLetters") // this starts the reaper actor and the user-configured logging subscribers, which are also actors registerOnTermination(stopScheduler()) - _locker = new Locker(scheduler, ReaperInterval, lookupRoot.path / "locker", deathWatch) + _locker = new Locker(scheduler, ReaperInterval, provider, lookupRoot.path / "locker", deathWatch) loadExtensions() if (LogConfigOnStart) logConfiguration() this diff --git a/akka-actor/src/main/scala/akka/actor/Locker.scala b/akka-actor/src/main/scala/akka/actor/Locker.scala index d4fd1badd5..ee6ea0c344 100644 --- a/akka-actor/src/main/scala/akka/actor/Locker.scala +++ b/akka-actor/src/main/scala/akka/actor/Locker.scala @@ -9,7 +9,12 @@ import akka.util.duration._ import java.util.concurrent.ConcurrentHashMap import akka.event.DeathWatch -class Locker(scheduler: Scheduler, period: Duration, val path: ActorPath, val deathWatch: DeathWatch) extends MinimalActorRef { +class Locker( + scheduler: Scheduler, + period: Duration, + val provider: ActorRefProvider, + val path: ActorPath, + val deathWatch: DeathWatch) extends MinimalActorRef { class DavyJones extends Runnable { def run = { diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index ccb5819074..8bc457904e 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -335,15 +335,15 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi import akka.pattern.ask MethodCall(method, args) match { case m if m.isOneWay ⇒ actor ! m; null //Null return value - case m if m.returnsFuture_? ⇒ actor.?(m, timeout) + case m if m.returnsFuture_? ⇒ ask(actor, m)(timeout) case m if m.returnsJOption_? || m.returnsOption_? ⇒ - val f = actor.?(m, timeout) + val f = ask(actor, m)(timeout) (try { Await.ready(f, timeout.duration).value } catch { case _: TimeoutException ⇒ None }) match { case None | Some(Right(null)) ⇒ if (m.returnsJOption_?) JOption.none[Any] else None case Some(Right(joption: AnyRef)) ⇒ joption case Some(Left(ex)) ⇒ throw ex } - case m ⇒ Await.result(actor.?(m, timeout), timeout.duration).asInstanceOf[AnyRef] + case m ⇒ Await.result(ask(actor, m)(timeout), timeout.duration).asInstanceOf[AnyRef] } } } diff --git a/akka-actor/src/main/scala/akka/event/Logging.scala b/akka-actor/src/main/scala/akka/event/Logging.scala index 854ac625ab..cb8f9df29e 100644 --- a/akka-actor/src/main/scala/akka/event/Logging.scala +++ b/akka-actor/src/main/scala/akka/event/Logging.scala @@ -649,6 +649,7 @@ object Logging { */ class StandardOutLogger extends MinimalActorRef with StdOutLogger { val path: ActorPath = new RootActorPath(LocalAddress("all-systems"), "/StandardOutLogger") + def provider: ActorRefProvider = throw new UnsupportedOperationException("StandardOutLogger does not provide") override val toString = "StandardOutLogger" override def !(message: Any)(implicit sender: ActorRef = null): Unit = print(message) } diff --git a/akka-actor/src/main/scala/akka/pattern/AskSupport.scala b/akka-actor/src/main/scala/akka/pattern/AskSupport.scala index bc79877fc3..ca07ea5052 100644 --- a/akka-actor/src/main/scala/akka/pattern/AskSupport.scala +++ b/akka-actor/src/main/scala/akka/pattern/AskSupport.scala @@ -51,7 +51,7 @@ object AskSupport { * * [see the [[akka.dispatch.Future]] companion object for a description of `flow`] */ - def ask(message: AnyRef)(implicit timeout: Timeout): Future[Any] = akka.pattern.ask(actorRef, message)(timeout) + def ask(message: Any)(implicit timeout: Timeout = null): Future[Any] = akka.pattern.ask(actorRef, message)(timeout) /** * Sends a message asynchronously and returns a [[akka.dispatch.Future]] @@ -81,20 +81,7 @@ object AskSupport { * * [see the [[akka.dispatch.Future]] companion object for a description of `flow`] */ - def ?(message: Any)(implicit timeout: Timeout): Future[Any] = akka.pattern.ask(actorRef, message) - - /* - * FIXME: I think this should be removed, since it introduces an “ambiguity” - * when sending Tuple2, which the compiler resolves unexpectedly to this - * method; also overloading is bad, isn’t it? - RK (ticket #1653) - */ - /** - * Sends a message asynchronously, returning a future which may eventually hold the reply. - * The implicit parameter with the default value is just there to disambiguate it from the version that takes the - * implicit timeout - */ - def ?(message: Any, timeout: Timeout)(implicit ignore: Int = 0): Future[Any] = ?(message)(timeout) - + def ?(message: Any)(implicit timeout: Timeout): Future[Any] = akka.pattern.ask(actorRef, message)(timeout) } /** @@ -102,6 +89,7 @@ object AskSupport { * receive the reply to an "ask" operation. */ private[akka] final class PromiseActorRef( + val provider: ActorRefProvider, val path: ActorPath, override val getParent: InternalActorRef, val result: Promise[Any], @@ -130,7 +118,7 @@ object AskSupport { def createAsker(provider: ActorRefProvider, timeout: Timeout): PromiseActorRef = { val path = provider.tempPath() val result = Promise[Any]()(provider.dispatcher) - val a = new PromiseActorRef(path, provider.tempContainer, result, provider.deathWatch) + val a = new PromiseActorRef(provider, path, provider.tempContainer, result, provider.deathWatch) provider.registerTempActor(a, path) val f = provider.scheduler.scheduleOnce(timeout.duration) { result.failure(new AskTimeoutException("Timed out")) } result onComplete { _ ⇒ diff --git a/akka-actor/src/main/scala/akka/pattern/package.scala b/akka-actor/src/main/scala/akka/pattern/package.scala index 6aaff00fd4..4abb7c41d7 100644 --- a/akka-actor/src/main/scala/akka/pattern/package.scala +++ b/akka-actor/src/main/scala/akka/pattern/package.scala @@ -3,6 +3,10 @@ */ package akka +import akka.actor._ +import akka.dispatch.{ Future, Promise } +import akka.util.{ Timeout, Duration } + /** * == Commonly Used Patterns With Akka == * @@ -38,10 +42,6 @@ package akka */ package object pattern { - import akka.actor._ - import akka.dispatch.{ Future, Promise } - import akka.util.{ Timeout, Duration } - /** * Import this implicit conversion to gain `?` and `ask` methods on * [[akka.actor.ActorRef]], which will defer to the @@ -88,12 +88,15 @@ package object pattern { * [see [[akka.dispatch.Future]] for a description of `flow`] */ def ask(actorRef: ActorRef, message: Any)(implicit timeout: Timeout = null): Future[Any] = actorRef match { - case ref: ActorRefWithProvider ⇒ + case ref: InternalActorRef if ref.isTerminated ⇒ + actorRef.tell(message) + Promise.failed(new AskTimeoutException("sending to terminated ref breaks promises"))(ref.provider.dispatcher) + case ref: InternalActorRef ⇒ val provider = ref.provider (if (timeout == null) provider.settings.ActorTimeout else timeout) match { case t if t.duration.length <= 0 ⇒ actorRef.tell(message) - Promise.failed(new AskTimeoutException("failed to create PromiseActorRef"))(provider.dispatcher) + Promise.failed(new AskTimeoutException("not asking with negative timeout"))(provider.dispatcher) case t ⇒ val a = AskSupport.createAsker(provider, t) actorRef.tell(message, a) @@ -132,7 +135,7 @@ package object pattern { def pipeTo[T](future: Future[T], actorRef: ActorRef): Future[T] = { future onComplete { case Right(r) ⇒ actorRef ! r - case Left(f) ⇒ actorRef ! akka.actor.Status.Failure(f) + case Left(f) ⇒ actorRef ! Status.Failure(f) } future } diff --git a/akka-actor/src/main/scala/akka/util/Duration.scala b/akka-actor/src/main/scala/akka/util/Duration.scala index 1406ad8564..1442d907d7 100644 --- a/akka-actor/src/main/scala/akka/util/Duration.scala +++ b/akka-actor/src/main/scala/akka/util/Duration.scala @@ -568,4 +568,3 @@ object Timeout { implicit def intToTimeout(timeout: Int) = new Timeout(timeout) implicit def longToTimeout(timeout: Long) = new Timeout(timeout) } - diff --git a/akka-agent/src/main/scala/akka/agent/Agent.scala b/akka-agent/src/main/scala/akka/agent/Agent.scala index cd19a0c781..8778d95760 100644 --- a/akka-agent/src/main/scala/akka/agent/Agent.scala +++ b/akka-agent/src/main/scala/akka/agent/Agent.scala @@ -124,7 +124,7 @@ class Agent[T](initialValue: T, system: ActorSystem) { * that new state can be obtained within the given timeout. */ def alter(f: T ⇒ T)(timeout: Timeout): Future[T] = { - def dispatch = updater.?(Alter(f), timeout).asInstanceOf[Future[T]] + def dispatch = ask(updater, Alter(f))(timeout).asInstanceOf[Future[T]] val txn = Txn.findCurrent if (txn.isDefined) { val result = Promise[T]()(system.dispatcher) @@ -172,7 +172,7 @@ class Agent[T](initialValue: T, system: ActorSystem) { send((value: T) ⇒ { suspend() val threadBased = system.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher("akka.agent.alter-off-dispatcher")) - result completeWith threadBased.?(Alter(f), timeout).asInstanceOf[Future[T]] + result completeWith ask(threadBased, Alter(f))(timeout).asInstanceOf[Future[T]] value }) result diff --git a/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala b/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala index bb82aab595..a753325429 100644 --- a/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala @@ -252,7 +252,7 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) { //#using-explicit-timeout import akka.util.duration._ import akka.pattern.ask - val future = myActor ? ("hello", timeout = 500 millis) + val future = myActor.ask("hello")(500 millis) //#using-explicit-timeout Await.result(future, 500 millis) must be("hello") } diff --git a/akka-remote/src/main/scala/akka/remote/Remote.scala b/akka-remote/src/main/scala/akka/remote/Remote.scala index f76aa8e908..628f9aea97 100644 --- a/akka-remote/src/main/scala/akka/remote/Remote.scala +++ b/akka-remote/src/main/scala/akka/remote/Remote.scala @@ -124,7 +124,7 @@ case class DaemonMsgWatch(watcher: ActorRef, watched: ActorRef) extends DaemonMs * It acts as the brain of the remote that responds to system remote events (messages) and undertakes action. */ class RemoteSystemDaemon(system: ActorSystemImpl, remote: Remote, _path: ActorPath, _parent: InternalActorRef, _log: LoggingAdapter) - extends VirtualPathContainer(_path, _parent, _log) { + extends VirtualPathContainer(system.provider, _path, _parent, _log) { /** * Find the longest matching path which we know about and return that ref diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index a04d37e207..388007ae33 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -166,7 +166,7 @@ private[akka] class RemoteActorRef private[akka] ( val path: ActorPath, val getParent: InternalActorRef, loader: Option[ClassLoader]) - extends InternalActorRef with RemoteRef with ActorRefWithProvider { + extends InternalActorRef with RemoteRef { def getChild(name: Iterator[String]): InternalActorRef = { val s = name.toStream From ef5287a6d87f974fc2267507abfcc2cad3100ee7 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 20 Jan 2012 17:38:58 +0100 Subject: [PATCH 074/152] Switching to LBD and minimizing window of failure --- .../main/scala/akka/actor/ActorSystem.scala | 56 +++++++------------ 1 file changed, 19 insertions(+), 37 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index b607bc6a7e..14d44106bb 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -4,7 +4,6 @@ package akka.actor import akka.config.ConfigurationException -import akka.actor._ import akka.event._ import akka.dispatch._ import akka.util.duration._ @@ -12,24 +11,15 @@ import akka.util.Timeout import akka.util.Timeout._ import org.jboss.netty.akka.util.HashedWheelTimer import java.util.concurrent.TimeUnit.MILLISECONDS -import java.util.concurrent.TimeUnit.NANOSECONDS -import java.io.File import com.typesafe.config.Config import com.typesafe.config.ConfigFactory -import com.typesafe.config.ConfigParseOptions -import com.typesafe.config.ConfigResolveOptions -import com.typesafe.config.ConfigException import akka.util.{ Helpers, Duration, ReflectiveAccess } -import java.util.concurrent.atomic.AtomicLong -import java.util.concurrent.{ CountDownLatch, Executors, ConcurrentHashMap } import scala.annotation.tailrec import org.jboss.netty.akka.util.internal.ConcurrentIdentityHashMap import java.io.Closeable -import java.util.concurrent.ConcurrentLinkedQueue import akka.dispatch.Await.Awaitable import akka.dispatch.Await.CanAwait -import java.util.concurrent.TimeUnit -import java.util.concurrent.TimeoutException +import java.util.concurrent.{ CountDownLatch, LinkedBlockingDeque, TimeUnit, TimeoutException } object ActorSystem { @@ -529,39 +519,31 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor override def toString = lookupRoot.path.root.address.toString class TerminationCallbacks extends Runnable with Awaitable[Unit] { - import scala.collection.JavaConverters._ - private val callbacks = new ConcurrentLinkedQueue[Runnable] + private val callbacks = new LinkedBlockingDeque[Runnable] private val latch = new CountDownLatch(1) - def add(callback: Runnable) { - callbacks add callback + final def add(callback: Runnable): Unit = + if (latch.await(0, TimeUnit.NANOSECONDS)) callbacks.clear() else callbacks addFirst callback + + final def run(): Unit = { + @tailrec def runNext(): Unit = callbacks.pollFirst() match { + case null ⇒ () + case some ⇒ + try some.run() catch { case e ⇒ log.error(e, "Failed to run termination callback, due to [{}]", e.getMessage) } + runNext() + } + try runNext() finally latch.countDown() } - def run(): Unit = try { - for (c ← callbacks.asScala.toSeq.reverse) { - try { - c.run() - } catch { - case e ⇒ - // TODO catching all and continue isn't good for OOME, ticket #1310 - log.error(e, "Failed to run termination callback, due to [{}]", e.getMessage) - } - } - } finally { - latch.countDown() - } + final def ready(atMost: Duration)(implicit permit: CanAwait): this.type = { + if (atMost.isFinite()) { + if (!latch.await(atMost.length, atMost.unit)) + throw new TimeoutException("Await termination timed out after [%s]" format (atMost.toString)) + } else latch.await() - def ready(atMost: Duration)(implicit permit: CanAwait) = { - if (atMost == Duration.Inf) { - latch.await() - } else { - val opened = latch.await(atMost.toNanos, TimeUnit.NANOSECONDS) - if (!opened) throw new TimeoutException("Await termination timed out after [%s]" format (atMost.toString)) - } this } - def result(atMost: Duration)(implicit permit: CanAwait): Unit = ready(atMost) - + final def result(atMost: Duration)(implicit permit: CanAwait): Unit = ready(atMost) } } From 1f99b425de2eaf91a143235bdf890b50dd26db2e Mon Sep 17 00:00:00 2001 From: Roland Date: Fri, 20 Jan 2012 18:09:26 +0100 Subject: [PATCH 075/152] docs for ask/pipeTo patterns --- .../docs/actor/UntypedActorDocTestBase.java | 50 +++++++++++++++++ akka-docs/java/untyped-actors.rst | 40 +++++++------- akka-docs/scala/actors.rst | 53 +++++++++---------- .../code/akka/docs/actor/ActorDocSpec.scala | 46 ++++++++-------- .../src/main/scala/akka/remote/Gossiper.scala | 2 +- 5 files changed, 119 insertions(+), 72 deletions(-) diff --git a/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java b/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java index 676254425e..054495b390 100644 --- a/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java +++ b/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java @@ -37,6 +37,17 @@ import akka.util.Duration; import akka.actor.ActorTimeoutException; //#import-gracefulStop +//#import-askPipeTo +import static akka.pattern.Patterns.ask; +import static akka.pattern.Patterns.pipeTo; +import akka.dispatch.Future; +import akka.dispatch.Futures; +import akka.util.Duration; +import akka.util.Timeout; +import java.util.concurrent.TimeUnit; +import java.util.ArrayList; +//#import-askPipeTo + import akka.actor.Props; import akka.actor.UntypedActor; import akka.actor.UntypedActorFactory; @@ -45,6 +56,8 @@ import akka.dispatch.MessageDispatcher; import org.junit.Test; import scala.Option; import java.lang.Object; +import java.util.ArrayList; +import java.util.Iterator; import java.util.concurrent.TimeUnit; import akka.pattern.Patterns; @@ -198,6 +211,43 @@ public class UntypedActorDocTestBase { //#gracefulStop system.shutdown(); } + + class Result { + final int x; + final String s; + public Result(int x, String s) { + this.x = x; + this.s = s; + } + } + + @Test + public void usePatternsAskPipeTo() { + ActorSystem system = ActorSystem.create("MySystem"); + ActorRef actorA = system.actorOf(new Props(MyUntypedActor.class)); + ActorRef actorB = system.actorOf(new Props(MyUntypedActor.class)); + ActorRef actorC = system.actorOf(new Props(MyUntypedActor.class)); + //#ask-pipeTo + final Timeout t = new Timeout(Duration.create(5, TimeUnit.SECONDS)); + + final ArrayList> futures = new ArrayList>(); + futures.add(ask(actorA, "request")); // using `akka.actor.timeout` from config + futures.add(ask(actorB, "reqeest", t)); // using explicit timeout from above + + final Future> aggregate = Futures.sequence(futures, system.dispatcher()); + + final Future transformed = aggregate.map(new akka.japi.Function, Result>() { + public Result apply(Iterable coll) { + final Iterator it = coll.iterator(); + final String s = (String) it.next(); + final int x = (Integer) it.next(); + return new Result(x, s); + } + }); + + pipeTo(transformed, actorC); + //#ask-pipeTo + } public static class MyActor extends UntypedActor { diff --git a/akka-docs/java/untyped-actors.rst b/akka-docs/java/untyped-actors.rst index 7e0d788590..b2978f7095 100644 --- a/akka-docs/java/untyped-actors.rst +++ b/akka-docs/java/untyped-actors.rst @@ -315,26 +315,37 @@ If invoked without the sender parameter the sender will be Ask: Send-And-Receive-Future ---------------------------- -Using ``?`` will send a message to the receiving Actor asynchronously and -will immediately return a :class:`Future` which will be completed with -an ``akka.actor.AskTimeoutException`` after the specified timeout: +The ``ask`` pattern involves actors as well as futures, hence it is offered as +a use pattern rather than a method on :class:`ActorRef`: -.. code-block:: java +.. includecode:: code/akka/docs/actor/UntypedActorDocTestBase.java#import-askPipeTo - long timeoutMillis = 1000; - Future future = actorRef.ask("Hello", timeoutMillis); +.. includecode:: code/akka/docs/actor/UntypedActorDocTestBase.java#ask-pipeTo -The receiving actor should reply to this message, which will complete the -future with the reply message as value; ``getSender.tell(result)``. +This example demonstrates ``ask`` together with the ``pipeTo`` pattern on +futures, because this is likely to be a common combination. Please note that +all of the above is completely non-blocking and asynchronous: ``ask`` produces +a :class:`Future`, two of which are composed into a new future using the +:meth:`Futures.sequence` and :meth:`map` methods and then ``pipeTo`` installs +an ``onComplete``-handler on the future to effect the submission of the +aggregated :class:`Result` to another actor. + +Using ``ask`` will send a message to the receiving Actor as with ``tell``, and +the receiving actor must reply with ``getSender().tell(reply)`` in order to +complete the returned :class:`Future` with a value. The ``ask`` operation +involves creating an internal actor for handling this reply, which needs to +have a timeout after which it is destroyed in order not to leak resources; see +more below. To complete the future with an exception you need send a Failure message to the sender. -This is not done automatically when an actor throws an exception while processing a +This is *not done automatically* when an actor throws an exception while processing a message. .. includecode:: code/akka/docs/actor/UntypedActorDocTestBase.java#reply-exception If the actor does not complete the future, it will expire after the timeout period, -specified as parameter to the ``ask`` method. +specified as parameter to the ``ask`` method; this will complete the +:class:`Future` with an :class:`AskTimeoutException`. See :ref:`futures-java` for more information on how to await or query a future. @@ -353,15 +364,6 @@ Gives you a way to avoid blocking. there is not yet a way to detect these illegal accesses at compile time. See also: :ref:`jmm-shared-state` -The future returned from the ``ask`` method can conveniently be passed around or -chained with further processing steps, but sometimes you just need the value, -even if that entails waiting for it (but keep in mind that waiting inside an -actor is prone to dead-locks, e.g. if obtaining the result depends on -processing another message on this actor). - -.. includecode:: code/akka/docs/actor/UntypedActorDocTestBase.java - :include: import-future,using-ask - Forward message --------------- diff --git a/akka-docs/scala/actors.rst b/akka-docs/scala/actors.rst index 558b50fac8..da0ae5b91e 100644 --- a/akka-docs/scala/actors.rst +++ b/akka-docs/scala/actors.rst @@ -325,9 +325,9 @@ Send messages Messages are sent to an Actor through one of the following methods. * ``!`` means “fire-and-forget”, e.g. send a message asynchronously and return - immediately. Also know as ``tell``. + immediately. Also known as ``tell``. * ``?`` sends a message asynchronously and returns a :class:`Future` - representing a possible reply. Also know as ``ask``. + representing a possible reply. Also known as ``ask``. Message ordering is guaranteed on a per-sender basis. @@ -352,25 +352,34 @@ If invoked from an instance that is **not** an Actor the sender will be Ask: Send-And-Receive-Future ---------------------------- -Using ``?`` will send a message to the receiving Actor asynchronously and -will immediately return a :class:`Future` which will be completed with -an ``akka.actor.AskTimeoutException`` after the specified timeout: +The ``ask`` pattern involves actors as well as futures, hence it is offered as +a use pattern rather than a method on :class:`ActorRef`: -.. code-block:: scala +.. includecode:: code/akka/docs/actor/ActorDocSpec.scala#ask-pipeTo - val future = actor ? "hello" +This example demonstrates ``ask`` together with the ``pipeTo`` pattern on +futures, because this is likely to be a common combination. Please note that +all of the above is completely non-blocking and asynchronous: ``ask`` produces +a :class:`Future`, three of which are composed into a new future using the +for-comprehension and then ``pipeTo`` installs an ``onComplete``-handler on the +future to effect the submission of the aggregated :class:`Result` to another +actor. -The receiving actor should reply to this message, which will complete the -future with the reply message as value; ``sender ! result``. +Using ``ask`` will send a message to the receiving Actor as with ``tell``, and +the receiving actor must reply with ``sender ! reply`` in order to complete the +returned :class:`Future` with a value. The ``ask`` operation involves creating +an internal actor for handling this reply, which needs to have a timeout after +which it is destroyed in order not to leak resources; see more below. To complete the future with an exception you need send a Failure message to the sender. -This is not done automatically when an actor throws an exception while processing a +This is *not done automatically* when an actor throws an exception while processing a message. .. includecode:: code/akka/docs/actor/ActorDocSpec.scala#reply-exception -If the actor does not complete the future, it will expire after the timeout period, -which is taken from one of the following locations in order of precedence: +If the actor does not complete the future, it will expire after the timeout +period, completing it with an :class:`AskTimeoutException`. The timeout is +taken from one of the following locations in order of precedence: 1. explicitly given timeout as in: @@ -380,6 +389,9 @@ which is taken from one of the following locations in order of precedence: .. includecode:: code/akka/docs/actor/ActorDocSpec.scala#using-implicit-timeout +3. actor system’s default value from ``akka.actor.timeout`` setting for + :meth:`ask` methods + See :ref:`futures-scala` for more information on how to await or query a future. @@ -398,23 +410,6 @@ Gives you a way to avoid blocking. there is not yet a way to detect these illegal accesses at compile time. See also: :ref:`jmm-shared-state` -The future returned from the ``?`` method can conveniently be passed around or -chained with further processing steps, but sometimes you just need the value, -even if that entails waiting for it (but keep in mind that waiting inside an -actor is prone to dead-locks, e.g. if obtaining the result depends on -processing another message on this actor). - -For this purpose, there is the method :meth:`Future.as[T]` which waits until -either the future is completed or its timeout expires, whichever comes first. -The result is then inspected and returned as :class:`Some[T]` if it was -normally completed and the answer’s runtime type matches the desired type; if -the future contains an exception or the value cannot be cast to the desired -type, it will throw the exception or a :class:`ClassCastException` (if you want -to get :obj:`None` in the latter case, use :meth:`Future.asSilently[T]`). In -case of a timeout, :obj:`None` is returned. - -.. includecode:: code/akka/docs/actor/ActorDocSpec.scala#using-ask - Forward message --------------- diff --git a/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala b/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala index a753325429..e2bef1776f 100644 --- a/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/actor/ActorDocSpec.scala @@ -212,28 +212,6 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) { system.stop(myActor) } - "using ask" in { - //#using-ask - import akka.pattern.ask - - class MyActor extends Actor { - def receive = { - case x: String ⇒ sender ! x.toUpperCase - case n: Int ⇒ sender ! (n + 1) - } - } - - val myActor = system.actorOf(Props(new MyActor), name = "myactor") - implicit val timeout = system.settings.ActorTimeout - val future = ask(myActor, "hello") - for (x ← future) println(x) //Prints "hello" - - val result: Future[Int] = for (x ← ask(myActor, 3).mapTo[Int]) yield { 2 * x } - //#using-ask - - system.stop(myActor) - } - "using implicit timeout" in { val myActor = system.actorOf(Props(new FirstActor)) //#using-implicit-timeout @@ -331,6 +309,28 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) { case e: ActorTimeoutException ⇒ // the actor wasn't stopped within 5 seconds } //#gracefulStop - } + + "using pattern ask / pipeTo" in { + val actorA, actorB, actorC, actorD = system.actorOf(Props.empty) + //#ask-pipeTo + import akka.pattern.{ ask, pipeTo } + + case class Result(x: Int, s: String, d: Double) + case object Request + + implicit val timeout = Timeout(5 seconds) // needed for `?` below + + val f: Future[Result] = + for { + x ← ask(actorA, Request).mapTo[Int] // call pattern directly + s ← actorB ask Request mapTo manifest[String] // call by implicit conversion + d ← actorC ? Request mapTo manifest[Double] // call by symbolic name + } yield Result(x, s, d) + + f pipeTo actorD // .. or .. + pipeTo(f, actorD) + //#ask-pipeTo + } + } diff --git a/akka-remote/src/main/scala/akka/remote/Gossiper.scala b/akka-remote/src/main/scala/akka/remote/Gossiper.scala index 788e86bb1c..74de988c59 100644 --- a/akka-remote/src/main/scala/akka/remote/Gossiper.scala +++ b/akka-remote/src/main/scala/akka/remote/Gossiper.scala @@ -252,7 +252,7 @@ class Gossiper(remote: Remote, system: ActorSystemImpl) { try { val t = remoteSettings.RemoteSystemDaemonAckTimeout - Await.result(connection ? (newGossip, t), t) match { + Await.result(connection.?(newGossip)(t), t) match { case Success(receiver) ⇒ log.debug("Gossip sent to [{}] was successfully received", receiver) case Failure(cause) ⇒ log.error(cause, cause.toString) } From f4587322be53b46de3be0987c7e774658d8c4975 Mon Sep 17 00:00:00 2001 From: Roland Date: Fri, 20 Jan 2012 18:34:09 +0100 Subject: [PATCH 076/152] =?UTF-8?q?make=20TestActorRef=E2=80=99s=20dispatc?= =?UTF-8?q?her=20configurable,=20fixes=20#1162?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../src/main/scala/akka/testkit/TestActorRef.scala | 4 +++- .../src/test/scala/akka/testkit/TestActorRefSpec.scala | 8 +++++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala index 35b5488ff5..1c1e6d03de 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala @@ -30,7 +30,9 @@ class TestActorRef[T <: Actor]( name: String) extends LocalActorRef( _system, - _props.withDispatcher(CallingThreadDispatcher.Id), + _props.withDispatcher( + if (_props.dispatcher == Dispatchers.DefaultDispatcherId) CallingThreadDispatcher.Id + else _props.dispatcher), _supervisor, _supervisor.path / name, false) { diff --git a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala index cd7e2307b4..ad5b572ec3 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala @@ -10,6 +10,7 @@ import akka.event.Logging.Warning import akka.dispatch.{ Future, Promise, Await } import akka.util.duration._ import akka.actor.ActorSystem +import akka.dispatch.Dispatcher /** * Test whether TestActorRef behaves as an ActorRef should, besides its own spec. @@ -88,7 +89,7 @@ object TestActorRefSpec { } @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) -class TestActorRefSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout { +class TestActorRefSpec extends AkkaSpec("disp1.type=Dispatcher") with BeforeAndAfterEach with DefaultTimeout { import TestActorRefSpec._ @@ -224,6 +225,11 @@ class TestActorRefSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTime a.underlying.dispatcher.getClass must be(classOf[CallingThreadDispatcher]) } + "allow override of dispatcher" in { + val a = TestActorRef(Props[WorkerActor].withDispatcher("disp1")) + a.underlying.dispatcher.getClass must be(classOf[Dispatcher]) + } + "proxy receive for the underlying actor" in { val ref = TestActorRef[WorkerActor] ref.receive("work") From 301b14e0ac3c407fd92efbcf9a4a3008ab4bb096 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 20 Jan 2012 18:38:48 +0100 Subject: [PATCH 077/152] Switching to a bit more robust approach --- .../scala/akka/actor/ActorSystemSpec.scala | 18 +++++--- .../main/scala/akka/actor/ActorSystem.scala | 46 ++++++++++++------- 2 files changed, 41 insertions(+), 23 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala index 25a4804c9c..9391ad43d1 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala @@ -6,11 +6,10 @@ package akka.actor import akka.testkit._ import org.scalatest.junit.JUnitSuite import com.typesafe.config.ConfigFactory -import java.util.concurrent.ConcurrentLinkedQueue -import java.util.concurrent.CountDownLatch import akka.dispatch.Await import akka.util.duration._ import scala.collection.JavaConverters +import java.util.concurrent.{ TimeUnit, RejectedExecutionException, CountDownLatch, ConcurrentLinkedQueue } class JavaExtensionSpec extends JavaExtension with JUnitSuite @@ -57,7 +56,7 @@ class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExt } - "awaitTtermination after termination callbacks" in { + "awaitTermination after termination callbacks" in { import scala.collection.JavaConverters._ val system2 = ActorSystem("AwaitTermination", AkkaSpec.testConf) @@ -69,13 +68,20 @@ class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExt callbackWasRun = true } - system2.scheduler.scheduleOnce(200.millis.dilated) { - system2.shutdown() - } + system2.scheduler.scheduleOnce(200.millis.dilated) { system2.shutdown() } system2.awaitTermination(5 seconds) callbackWasRun must be(true) + } + "throw RejectedExecutionException when shutdown" in { + val system2 = ActorSystem("AwaitTermination", AkkaSpec.testConf) + system2.shutdown() + system2.awaitTermination(5 seconds) + + intercept[RejectedExecutionException] { + system2.registerOnTermination { println("IF YOU SEE THIS THEN THERE'S A BUG HERE") } + }.getMessage must be("Must be called prior to system shutdown.") } } diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index 14d44106bb..447e6cd5a2 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -7,19 +7,19 @@ import akka.config.ConfigurationException import akka.event._ import akka.dispatch._ import akka.util.duration._ -import akka.util.Timeout import akka.util.Timeout._ import org.jboss.netty.akka.util.HashedWheelTimer import java.util.concurrent.TimeUnit.MILLISECONDS import com.typesafe.config.Config import com.typesafe.config.ConfigFactory -import akka.util.{ Helpers, Duration, ReflectiveAccess } 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 java.util.concurrent.{ CountDownLatch, LinkedBlockingDeque, TimeUnit, TimeoutException } +import java.util.concurrent.{ CountDownLatch, TimeoutException, RejectedExecutionException } +import akka.util._ +import collection.immutable.Stack object ActorSystem { @@ -224,7 +224,9 @@ abstract class ActorSystem extends ActorRefFactory { * The callbacks will be run sequentilly in reverse order of registration, i.e. * last registration is run first. * - * Callbacks registered after that the shutdown process has started will likely not be run. + * @throws a RejectedExecutionException if the System has already shut down or if shutdown has been initiated. + * + * Scala API */ def registerOnTermination[T](code: ⇒ T): Unit @@ -234,7 +236,7 @@ abstract class ActorSystem extends ActorRefFactory { * The callbacks will be run sequentilly in reverse order of registration, i.e. * last registration is run first. * - * Callbacks registered after that the shutdown process has started will likely not be run. + * @throws a RejectedExecutionException if the System has already shut down or if shutdown has been initiated. * * Java API */ @@ -518,21 +520,31 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor override def toString = lookupRoot.path.root.address.toString - class TerminationCallbacks extends Runnable with Awaitable[Unit] { - private val callbacks = new LinkedBlockingDeque[Runnable] + final class TerminationCallbacks extends Runnable with Awaitable[Unit] { + private val lock = new ReentrantGuard + private var callbacks: Stack[Runnable] = _ //non-volatile since guarded by the lock + lock withGuard { callbacks = Stack.empty[Runnable] } + private val latch = new CountDownLatch(1) - final def add(callback: Runnable): Unit = - if (latch.await(0, TimeUnit.NANOSECONDS)) callbacks.clear() else callbacks addFirst callback - - final def run(): Unit = { - @tailrec def runNext(): Unit = callbacks.pollFirst() match { - case null ⇒ () - case some ⇒ - try some.run() catch { case e ⇒ log.error(e, "Failed to run termination callback, due to [{}]", e.getMessage) } - runNext() + final def add(callback: Runnable): Unit = { + latch.getCount match { + case 0 ⇒ throw new RejectedExecutionException("Must be called prior to system shutdown.") + case _ ⇒ lock withGuard { + if (latch.getCount == 0) throw new RejectedExecutionException("Must be called prior to system shutdown.") + else callbacks = callbacks.push(callback) + } } - try runNext() finally latch.countDown() + } + + final def run(): Unit = lock withGuard { + @tailrec def runNext(c: Stack[Runnable]): Stack[Runnable] = c.headOption match { + case None ⇒ Stack.empty[Runnable] + case Some(callback) ⇒ + try callback.run() catch { case e ⇒ log.error(e, "Failed to run termination callback, due to [{}]", e.getMessage) } + runNext(c.pop) + } + try { callbacks = runNext(callbacks) } finally latch.countDown() } final def ready(atMost: Duration)(implicit permit: CanAwait): this.type = { From f6d71881ea74bdce9ce75e334070fb2a586b497f Mon Sep 17 00:00:00 2001 From: Roland Date: Fri, 20 Jan 2012 19:45:23 +0100 Subject: [PATCH 078/152] add ?(msg, timeout) migration helper, see #1653 --- .../akka/migration/AskableActorRef.scala | 74 +++++++++++++++++++ .../main/scala/akka/migration/package.scala | 3 + .../main/scala/akka/pattern/AskSupport.scala | 2 +- .../project/migration-guide-1.3.x-2.0.x.rst | 20 +++++ 4 files changed, 98 insertions(+), 1 deletion(-) create mode 100644 akka-actor-migration/src/main/scala/akka/migration/AskableActorRef.scala diff --git a/akka-actor-migration/src/main/scala/akka/migration/AskableActorRef.scala b/akka-actor-migration/src/main/scala/akka/migration/AskableActorRef.scala new file mode 100644 index 0000000000..caf8921812 --- /dev/null +++ b/akka-actor-migration/src/main/scala/akka/migration/AskableActorRef.scala @@ -0,0 +1,74 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.migration + +import akka.actor.ActorRef +import akka.dispatch.Future +import akka.util.Timeout + +class AskableActorRef(val actorRef: ActorRef) { + + /** + * 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 + * given timeout has expired; this is independent from any timeout applied + * while awaiting a result for this future (i.e. in + * `Await.result(..., timeout)`). + * + * Warning: + * When using future callbacks, inside actors you need to carefully avoid closing over + * the containing actor’s object, i.e. do not call methods or access mutable state + * on the enclosing actor from within the callback. This would break the actor + * encapsulation and may introduce synchronization bugs and race conditions because + * the callback will be scheduled concurrently to the enclosing actor. Unfortunately + * there is not yet a way to detect these illegal accesses at compile time. + * + * Recommended usage: + * + * {{{ + * val f = worker.ask(request)(timeout) + * flow { + * EnrichedRequest(request, f()) + * } pipeTo nextActor + * }}} + * + * [see the [[akka.dispatch.Future]] companion object for a description of `flow`] + */ + def ask(message: Any)(implicit timeout: Timeout = null): Future[Any] = akka.pattern.ask(actorRef, message)(timeout) + + /** + * 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 + * given timeout has expired; this is independent from any timeout applied + * while awaiting a result for this future (i.e. in + * `Await.result(..., timeout)`). + * + * Warning: + * When using future callbacks, inside actors you need to carefully avoid closing over + * the containing actor’s object, i.e. do not call methods or access mutable state + * on the enclosing actor from within the callback. This would break the actor + * encapsulation and may introduce synchronization bugs and race conditions because + * the callback will be scheduled concurrently to the enclosing actor. Unfortunately + * there is not yet a way to detect these illegal accesses at compile time. + * + * Recommended usage: + * + * {{{ + * val f = worker ? request + * flow { + * EnrichedRequest(request, f()) + * } pipeTo nextActor + * }}} + * + * [see the [[akka.dispatch.Future]] companion object for a description of `flow`] + */ + def ?(message: Any)(implicit timeout: Timeout): Future[Any] = akka.pattern.ask(actorRef, message)(timeout) + + @deprecated("use ?(msg)(timeout), this method has dangerous ambiguity", "2.0-migration") + def ?(message: Any, timeout: Timeout)(i: Int = 0): Future[Any] = this.?(message)(timeout) +} \ No newline at end of file diff --git a/akka-actor-migration/src/main/scala/akka/migration/package.scala b/akka-actor-migration/src/main/scala/akka/migration/package.scala index 319fdd997e..4fd8d5eeea 100644 --- a/akka-actor-migration/src/main/scala/akka/migration/package.scala +++ b/akka-actor-migration/src/main/scala/akka/migration/package.scala @@ -31,4 +31,7 @@ package object migration { def stop(): Unit = GlobalActorSystem.stop(actorRef) } + implicit def ask(actorRef: ActorRef) = new akka.migration.AskableActorRef(actorRef) + def ask(actorRef: ActorRef, message: Any)(implicit timeout: Timeout = null): Future[Any] = akka.pattern.ask(actorRef, message)(timeout) + } \ No newline at end of file diff --git a/akka-actor/src/main/scala/akka/pattern/AskSupport.scala b/akka-actor/src/main/scala/akka/pattern/AskSupport.scala index ca07ea5052..57dec1debe 100644 --- a/akka-actor/src/main/scala/akka/pattern/AskSupport.scala +++ b/akka-actor/src/main/scala/akka/pattern/AskSupport.scala @@ -81,7 +81,7 @@ object AskSupport { * * [see the [[akka.dispatch.Future]] companion object for a description of `flow`] */ - def ?(message: Any)(implicit timeout: Timeout): Future[Any] = akka.pattern.ask(actorRef, message)(timeout) + def ?(message: Any)(implicit timeout: Timeout = null): Future[Any] = akka.pattern.ask(actorRef, message)(timeout) } /** diff --git a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst index a3f874894d..33752b8ce7 100644 --- a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst +++ b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst @@ -205,6 +205,26 @@ Documentation: * :ref:`actors-scala` * :ref:`untyped-actors-java` +``ActorRef.?(msg, timeout)`` +^^^^^^^^^^^^^^^^^^^^^^^^^^^^ + +This method has a dangerous overlap with ``ActorRef.?(msg)(implicit timeout)`` +due to the fact that Scala allows to pass a :class:`Tuple` in place of the +message without requiring extra parentheses:: + + actor ? (1, "hallo") // will send a tuple + actor ? (1, Timeout()) // will send 1 with an explicit timeout + +To remove this ambiguity, the latter variant is removed in version 2.0. If you +were using it before, it will now send tuples where that is not desired. In +order to correct all places in the code where this happens, simply import +``akka.migration.ask`` instead of ``akka.pattern.ask`` to obtain a variant +which will give deprecation warnings where the old method signature is used:: + + import akka.migration.ask + + actor ? (1, Timeout(2 seconds)) // will give deprecation warning + ActorPool ^^^^^^^^^ From 1333700c0dffec830534fc2baa69504b8f6e213d Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 20 Jan 2012 19:52:50 +0100 Subject: [PATCH 079/152] Unborkening master --- akka-docs/scala/code/akka/docs/actor/TypedActorDocSpec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-docs/scala/code/akka/docs/actor/TypedActorDocSpec.scala b/akka-docs/scala/code/akka/docs/actor/TypedActorDocSpec.scala index da718b503d..b1fcc9224c 100644 --- a/akka-docs/scala/code/akka/docs/actor/TypedActorDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/actor/TypedActorDocSpec.scala @@ -45,7 +45,7 @@ class SquarerImpl(val name: String) extends Squarer { //#typed-actor-impl-methods } //#typed-actor-impl -import java.lang.Integer.{ parseInt ⇒ println } //Mr funny man avoids printing to stdout AND keeping docs alright +import java.lang.String.{ valueOf ⇒ println } //Mr funny man avoids printing to stdout AND keeping docs alright //#typed-actor-supercharge trait Foo { def doFoo(times: Int): Unit = println("doFoo(" + times + ")") From 9b8c39bc01718eb6d3d7ad0442c4a4510ec45313 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Fri, 20 Jan 2012 12:58:57 -0700 Subject: [PATCH 080/152] More scaladocs for IO --- akka-actor/src/main/scala/akka/actor/IO.scala | 157 ++++++++++++++---- 1 file changed, 129 insertions(+), 28 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/IO.scala b/akka-actor/src/main/scala/akka/actor/IO.scala index 50574ef662..176501d5b8 100644 --- a/akka-actor/src/main/scala/akka/actor/IO.scala +++ b/akka-actor/src/main/scala/akka/actor/IO.scala @@ -481,6 +481,9 @@ object IO { Cont(step(ByteString.empty)) } + /** + * An Iteratee that will collect bytes as long as a predicate is true. + */ def takeWhile(p: (Byte) ⇒ Boolean): Iteratee[ByteString] = { def step(taken: ByteString)(input: Input): (Iteratee[ByteString], Input) = input match { case Chunk(more) ⇒ @@ -556,6 +559,9 @@ object IO { case eof @ EOF(cause) ⇒ (Cont(more ⇒ (Done(ByteString.empty), more), cause), eof) } + /** + * An Iteratee that creates a list made up of the results of an Iteratee. + */ def takeList[A](length: Int)(iter: Iteratee[A]): Iteratee[List[A]] = { def step(left: Int, list: List[A]): Iteratee[List[A]] = if (left == 0) Done(list.reverse) @@ -564,6 +570,10 @@ object IO { step(length, Nil) } + /** + * An Iteratee that returns a [[akka.util.ByteString]] of the request length, + * but does not consume the Input. + */ def peek(length: Int): Iteratee[ByteString] = { def step(taken: ByteString)(input: Input): (Iteratee[ByteString], Input) = input match { case Chunk(more) ⇒ @@ -579,12 +589,25 @@ object IO { Cont(step(ByteString.empty)) } + /** + * An Iteratee that continually repeats an Iteratee. + * + * TODO: Should terminate on EOF + */ def repeat(iter: Iteratee[Unit]): Iteratee[Unit] = iter flatMap (_ ⇒ repeat(iter)) + /** + * An Iteratee that applies an Iteratee to each element of a Traversable + * and finally returning a single Iteratee containing a Traversable of the results. + */ def traverse[A, B, M[A] <: Traversable[A]](in: M[A])(f: A ⇒ Iteratee[B])(implicit cbf: CanBuildFrom[M[A], B, M[B]]): Iteratee[M[B]] = fold(cbf(in), in)((b, a) ⇒ f(a) map (b += _)) map (_.result) + /** + * An Iteratee that folds over a Traversable by applying a function that + * returns an Iteratee. + */ def fold[A, B, M[A] <: Traversable[A]](initial: B, in: M[A])(f: (B, A) ⇒ Iteratee[B]): Iteratee[B] = (Iteratee(initial) /: in)((ib, a) ⇒ ib flatMap (b ⇒ f(b, a))) @@ -629,24 +652,85 @@ object IO { } +/** + * IOManager contains a reference to the [[akka.actor.IOManagerActor]] for + * an [[akka.actor.ActorSystem]]. + * + * This is the recommended entry point to creating sockets for performing + * IO. + * + * Use the companion object to retrieve the instance of this class for an + * ActorSystem. + * + * {{{ + * val ioManager = IOManager(context.system) + * val socket = ioManager.connect("127.0.0.1") + * }}} + * + * 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 { + /** + * A reference to the [[akka.actor.IOManagerActor]] that performs the actual + * IO. It communicates with other actors using subclasses of + * [[akka.actor.IO.IOMessage]]. + */ val actor = system.actorOf(Props[IOManagerActor], "io-manager") + /** + * Create a ServerSocketChannel listening on an address. Messages will be + * sent from the [[akka.actor.IOManagerActor]] to the owner + * [[akka.actor.ActorRef]]. + * + * @param address the address to listen on + * @param owner the ActorRef that will receive messages from the IOManagerActor + * @return a [[akka.actor.IO.ServerHandle]] to uniquely identify the created socket + */ def listen(address: InetSocketAddress)(implicit owner: ActorRef): IO.ServerHandle = { val server = IO.ServerHandle(owner, actor) actor ! IO.Listen(server, address) server } + /** + * Create a ServerSocketChannel listening on a host and port. Messages will + * be sent from the [[akka.actor.IOManagerActor]] to the owner + * [[akka.actor.ActorRef]]. + * + * @param host the hostname or IP to listen on + * @param port the port to listen on + * @param owner the ActorRef that will receive messages from the IOManagerActor + * @return a [[akka.actor.IO.ServerHandle]] to uniquely identify the created socket + */ def listen(host: String, port: Int)(implicit owner: ActorRef): IO.ServerHandle = listen(new InetSocketAddress(host, port))(owner) + /** + * Create a SocketChannel connecting to an address. Messages will be + * sent from the [[akka.actor.IOManagerActor]] to the owner + * [[akka.actor.ActorRef]]. + * + * @param address the address to connect to + * @param owner the ActorRef that will receive messages from the IOManagerActor + * @return a [[akka.actor.IO.SocketHandle]] to uniquely identify the created socket + */ def connect(address: InetSocketAddress)(implicit owner: ActorRef): IO.SocketHandle = { val socket = IO.SocketHandle(owner, actor) actor ! IO.Connect(socket, address) socket } + /** + * Create a SocketChannel connecting to a host and port. Messages will + * be sent from the [[akka.actor.IOManagerActor]] to the owner + * [[akka.actor.ActorRef]]. + * + * @param host the hostname or IP to connect to + * @param port the port to connect to + * @param owner the ActorRef that will receive messages from the IOManagerActor + * @return a [[akka.actor.IO.SocketHandle]] to uniquely identify the created socket + */ def connect(host: String, port: Int)(implicit owner: ActorRef): IO.SocketHandle = connect(new InetSocketAddress(host, port))(owner) @@ -657,40 +741,57 @@ object IOManager extends ExtensionId[IOManager] with ExtensionIdProvider { override def createExtension(system: ActorSystemImpl) = new IOManager(system) } -// TODO: Support a pool of workers +/** + * An [[akka.actor.Actor]] that performs IO using a Java NIO Selector. + * + * Use [[akka.actor.IOManager]] to retrieve an instance of this Actor. + */ final class IOManagerActor extends Actor { import SelectionKey.{ OP_READ, OP_WRITE, OP_ACCEPT, OP_CONNECT } - val bufferSize = 8192 // TODO: make buffer size configurable + private val bufferSize = 8192 // TODO: make buffer size configurable - type ReadChannel = ReadableByteChannel with SelectableChannel - type WriteChannel = WritableByteChannel with SelectableChannel + private type ReadChannel = ReadableByteChannel with SelectableChannel + private type WriteChannel = WritableByteChannel with SelectableChannel - val selector: Selector = Selector open () + private val selector: Selector = Selector open () - val channels = mutable.Map.empty[IO.Handle, SelectableChannel] + private val channels = mutable.Map.empty[IO.Handle, SelectableChannel] - val accepted = mutable.Map.empty[IO.ServerHandle, mutable.Queue[SelectableChannel]] + private val accepted = mutable.Map.empty[IO.ServerHandle, mutable.Queue[SelectableChannel]] - val writes = mutable.Map.empty[IO.WriteHandle, WriteBuffer] + private val writes = mutable.Map.empty[IO.WriteHandle, WriteBuffer] - val closing = mutable.Set.empty[IO.Handle] + /** Channels that should close after writes are complete */ + private val closing = mutable.Set.empty[IO.Handle] - val buffer = ByteBuffer.allocate(bufferSize) + /** Buffer used for all reads */ + private val buffer = ByteBuffer.allocate(bufferSize) - var lastSelect = 0 + /** a counter that is incremented each time a message is retreived */ + private var lastSelect = 0 - val selectAt = 100 // TODO: determine best value, perhaps based on throughput? Other triggers (like write queue size)? + /** force a select when lastSelect reaches this amount */ + private val selectAt = 100 - var running = false + /** true while the selector is open and channels.nonEmpty */ + private var running = false - var selectSent = false + /** is there already a Select message in flight? */ + private var selectSent = false - var fastSelect = false + /** + * select blocks for 1ms when false and is completely nonblocking when true. + * Automatically changes due to activity. This reduces object allocations + * when there are no pending events. + */ + private var fastSelect = false - object Select + /** unique message that is sent to ourself to initiate the next select */ + private object Select - def run() { + /** This method should be called after receiving any message */ + private def run() { if (!running) { running = true if (!selectSent) { @@ -702,7 +803,7 @@ final class IOManagerActor extends Actor { if (lastSelect >= selectAt) select() } - def select() { + private def select() { if (selector.isOpen) { // TODO: Make select behaviour configurable. // Blocking 1ms reduces allocations during idle times, non blocking gives better performance. @@ -721,7 +822,7 @@ final class IOManagerActor extends Actor { lastSelect = 0 } - def receive = { + protected def receive = { case Select ⇒ select() if (running) self ! Select @@ -785,7 +886,7 @@ final class IOManagerActor extends Actor { selector.close } - def process(key: SelectionKey) { + private def process(key: SelectionKey) { val handle = key.attachment.asInstanceOf[IO.Handle] try { if (key.isConnectable) key.channel match { @@ -814,7 +915,7 @@ final class IOManagerActor extends Actor { } } - def cleanup(handle: IO.Handle, cause: Option[Exception]) { + private def cleanup(handle: IO.Handle, cause: Option[Exception]) { closing -= handle handle match { case server: IO.ServerHandle ⇒ accepted -= server @@ -829,22 +930,22 @@ final class IOManagerActor extends Actor { } } - def setOps(handle: IO.Handle, ops: Int): Unit = + private def setOps(handle: IO.Handle, ops: Int): Unit = channels(handle) keyFor selector interestOps ops - def addOps(handle: IO.Handle, ops: Int) { + private def addOps(handle: IO.Handle, ops: Int) { val key = channels(handle) keyFor selector val cur = key.interestOps key interestOps (cur | ops) } - def removeOps(handle: IO.Handle, ops: Int) { + private def removeOps(handle: IO.Handle, ops: Int) { val key = channels(handle) keyFor selector val cur = key.interestOps key interestOps (cur - (cur & ops)) } - def connect(socket: IO.SocketHandle, channel: SocketChannel) { + private def connect(socket: IO.SocketHandle, channel: SocketChannel) { if (channel.finishConnect) { removeOps(socket, OP_CONNECT) socket.owner ! IO.Connected(socket) @@ -854,7 +955,7 @@ final class IOManagerActor extends Actor { } @tailrec - def accept(server: IO.ServerHandle, channel: ServerSocketChannel) { + private def accept(server: IO.ServerHandle, channel: ServerSocketChannel) { val socket = channel.accept if (socket ne null) { socket configureBlocking false @@ -874,7 +975,7 @@ final class IOManagerActor extends Actor { } @tailrec - def read(handle: IO.ReadHandle, channel: ReadChannel) { + private def read(handle: IO.ReadHandle, channel: ReadChannel) { buffer.clear val readLen = channel read buffer if (readLen == -1) { @@ -886,7 +987,7 @@ final class IOManagerActor extends Actor { } } - def write(handle: IO.WriteHandle, channel: WriteChannel) { + private def write(handle: IO.WriteHandle, channel: WriteChannel) { val queue = writes(handle) queue write channel if (queue.isEmpty) { From d5616e123beb54c5127f504b1d4d7064887d9d2a Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Fri, 20 Jan 2012 13:38:22 -0700 Subject: [PATCH 081/152] Scaladoc for ByteString --- .../src/main/scala/akka/util/ByteString.scala | 82 ++++++++++++++++++- 1 file changed, 81 insertions(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/util/ByteString.scala b/akka-actor/src/main/scala/akka/util/ByteString.scala index fb41df2429..125b605a89 100644 --- a/akka-actor/src/main/scala/akka/util/ByteString.scala +++ b/akka-actor/src/main/scala/akka/util/ByteString.scala @@ -10,27 +10,49 @@ import scala.collection.generic.{ CanBuildFrom, GenericCompanion } object ByteString { + /** + * Creates a new ByteString by copying a byte array. + */ def apply(bytes: Array[Byte]): ByteString = ByteString1(bytes.clone) + /** + * Creates a new ByteString by copying bytes. + */ def apply(bytes: Byte*): ByteString = { val ar = new Array[Byte](bytes.size) bytes.copyToArray(ar) ByteString1(ar) } + /** + * Creates a new ByteString by converting from integral numbers to bytes. + */ def apply[T](bytes: T*)(implicit num: Integral[T]): ByteString = ByteString1(bytes.map(x ⇒ num.toInt(x).toByte)(collection.breakOut)) + /** + * Creates a new ByteString by copying bytes from a ByteBuffer. + */ def apply(bytes: ByteBuffer): ByteString = { val ar = new Array[Byte](bytes.remaining) bytes.get(ar) ByteString1(ar) } + /** + * Creates a new ByteString by encoding a String as UTF-8. + */ def apply(string: String): ByteString = apply(string, "UTF-8") + /** + * Creates a new ByteString by encoding a String with a charset. + */ def apply(string: String, charset: String): ByteString = ByteString1(string.getBytes(charset)) + /** + * Creates a new ByteString by copying length bytes starting at offset from + * an Array. + */ def fromArray(array: Array[Byte], offset: Int, length: Int): ByteString = { val copyOffset = math.max(offset, 0) val copyLength = math.max(math.min(array.length - copyOffset, length), 0) @@ -55,6 +77,9 @@ object ByteString { def apply(bytes: Array[Byte]) = new ByteString1(bytes) } + /** + * An unfragmented ByteString. + */ final class ByteString1 private (private val bytes: Array[Byte], private val startIndex: Int, val length: Int) extends ByteString { private def this(bytes: Array[Byte]) = this(bytes, 0, bytes.length) @@ -152,6 +177,9 @@ object ByteString { } + /** + * A ByteString with 2 or more fragments. + */ final class ByteStrings private (val bytestrings: Vector[ByteString1], val length: Int) extends ByteString { def apply(idx: Int): Byte = @@ -233,18 +261,70 @@ object ByteString { } -sealed trait ByteString extends IndexedSeq[Byte] with IndexedSeqOptimized[Byte, ByteString] { +/** + * A [[http://en.wikipedia.org/wiki/Rope_(computer_science) Rope-like]] immutable + * data structure containing bytes. The goal of this structure is to reduce + * copying of arrays when concatenating and slicing sequences of bytes, and also + * providing a thread safe way of working with bytes. + * + * TODO: Add performance characteristics + */ +abstract class ByteString extends IndexedSeq[Byte] with IndexedSeqOptimized[Byte, ByteString] { override protected[this] def newBuilder = ByteString.newBuilder + + /** + * Efficiently concatenate another ByteString. + */ def ++(that: ByteString): ByteString + + /** + * Copy as many bytes as possible to a ByteBuffer, starting from it's + * current position. This method will not overflow the buffer. + * + * @param buffer a ByteBuffer to copy bytes to + * @return the number of bytes actually copied + */ def copyToBuffer(buffer: ByteBuffer): Int + + /** + * Create a new ByteString with all contents compacted into a single + * byte array. + */ def compact: ByteString + + /** + * Returns a read-only ByteBuffer that directly wraps this ByteString + * if it is not fragmented. + */ def asByteBuffer: ByteBuffer + + /** + * Creates a new ByteBuffer with a copy of all bytes contained in this + * ByteString. + */ final def toByteBuffer: ByteBuffer = ByteBuffer.wrap(toArray) + + /** + * Decodes this ByteString as a UTF-8 encoded String. + */ final def utf8String: String = decodeString("UTF-8") + + /** + * Decodes this ByteString using a charset to produce a String. + */ def decodeString(charset: String): String + + /** + * map method that will automatically cast Int back into Byte. + */ final def mapI(f: Byte ⇒ Int): ByteString = map(f andThen (_.toByte)) } +/** + * A mutable builder for efficiently creating a [[akka.util.ByteString]]. + * + * The created ByteString is not automatically compacted. + */ final class ByteStringBuilder extends Builder[Byte, ByteString] { import ByteString.{ ByteString1, ByteStrings } private var _length = 0 From 7e0cf29e8f322349cdb3f04c4f188580903c5581 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sat, 21 Jan 2012 00:32:37 +0100 Subject: [PATCH 082/152] Adding config options, tests and conditional code for remote message logging --- akka-remote/src/main/resources/reference.conf | 6 ++ .../src/main/scala/akka/remote/Remote.scala | 3 +- .../scala/akka/remote/RemoteSettings.scala | 60 ++++++++++--------- .../remote/netty/NettyRemoteSupport.scala | 4 +- .../scala/akka/remote/RemoteConfigSpec.scala | 3 + 5 files changed, 45 insertions(+), 31 deletions(-) diff --git a/akka-remote/src/main/resources/reference.conf b/akka-remote/src/main/resources/reference.conf index dee44bd86c..4db13bb264 100644 --- a/akka-remote/src/main/resources/reference.conf +++ b/akka-remote/src/main/resources/reference.conf @@ -59,6 +59,12 @@ akka { # Reuse inbound connections for outbound messages use-passive-connections = on + # If this is "on", Akka will log all inbound messages at DEBUG level, if off then they are not logged + log-received-messages = off + + # If this is "on", Akka will log all outbound messages at DEBUG level, if off then they are not logged + log-sent-messages = off + # accrual failure detection config failure-detector { diff --git a/akka-remote/src/main/scala/akka/remote/Remote.scala b/akka-remote/src/main/scala/akka/remote/Remote.scala index c5535833aa..04b1bf83a4 100644 --- a/akka-remote/src/main/scala/akka/remote/Remote.scala +++ b/akka-remote/src/main/scala/akka/remote/Remote.scala @@ -239,7 +239,8 @@ trait RemoteMarshallingOps { } def receiveMessage(remoteMessage: RemoteMessage) { - log.debug("received message {}", remoteMessage) + if (remote.remoteSettings.LogReceivedMessages) + log.debug("received message {}", remoteMessage) val remoteDaemon = remote.remoteDaemon diff --git a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala index b861026e9d..8a9a55a679 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala @@ -15,22 +15,24 @@ class RemoteSettings(val config: Config, val systemName: String) { import config._ - val RemoteTransport = getString("akka.remote.transport") - val FailureDetectorThreshold = getInt("akka.remote.failure-detector.threshold") - val FailureDetectorMaxSampleSize = getInt("akka.remote.failure-detector.max-sample-size") - val ShouldCompressData = getBoolean("akka.remote.use-compression") - val RemoteSystemDaemonAckTimeout = Duration(getMilliseconds("akka.remote.remote-daemon-ack-timeout"), MILLISECONDS) - val InitalDelayForGossip = Duration(getMilliseconds("akka.remote.gossip.initialDelay"), MILLISECONDS) - val GossipFrequency = Duration(getMilliseconds("akka.remote.gossip.frequency"), MILLISECONDS) - val BackoffTimeout = Duration(getMilliseconds("akka.remote.backoff-timeout"), MILLISECONDS) + final val RemoteTransport = getString("akka.remote.transport") + final val FailureDetectorThreshold = getInt("akka.remote.failure-detector.threshold") + final val FailureDetectorMaxSampleSize = getInt("akka.remote.failure-detector.max-sample-size") + final val ShouldCompressData = getBoolean("akka.remote.use-compression") + final val RemoteSystemDaemonAckTimeout = Duration(getMilliseconds("akka.remote.remote-daemon-ack-timeout"), MILLISECONDS) + final val InitalDelayForGossip = Duration(getMilliseconds("akka.remote.gossip.initialDelay"), MILLISECONDS) + final val GossipFrequency = Duration(getMilliseconds("akka.remote.gossip.frequency"), MILLISECONDS) + final val BackoffTimeout = Duration(getMilliseconds("akka.remote.backoff-timeout"), MILLISECONDS) + final val LogReceivedMessages = getBoolean("akka.remote.log-received-messages") + final val LogSentMessages = getBoolean("akka.remote.log-sent-messages") // TODO cluster config will go into akka-cluster/reference.conf when we enable that module - val SeedNodes = Set.empty[RemoteNettyAddress] ++ getStringList("akka.cluster.seed-nodes").asScala.collect { + final val SeedNodes = Set.empty[RemoteNettyAddress] ++ getStringList("akka.cluster.seed-nodes").asScala.collect { case RemoteAddressExtractor(addr) ⇒ addr.transport } - val serverSettings = new RemoteServerSettings - val clientSettings = new RemoteClientSettings + final val serverSettings = new RemoteServerSettings + final val clientSettings = new RemoteClientSettings class RemoteClientSettings { val SecureCookie: Option[String] = getString("akka.remote.secure-cookie") match { @@ -38,35 +40,35 @@ class RemoteSettings(val config: Config, val systemName: String) { case cookie ⇒ Some(cookie) } - val ConnectionTimeout = Duration(getMilliseconds("akka.remote.client.connection-timeout"), MILLISECONDS) - val ReconnectionTimeWindow = Duration(getMilliseconds("akka.remote.client.reconnection-time-window"), MILLISECONDS) - val ReadTimeout = Duration(getMilliseconds("akka.remote.client.read-timeout"), MILLISECONDS) - val ReconnectDelay = Duration(getMilliseconds("akka.remote.client.reconnect-delay"), MILLISECONDS) - val MessageFrameSize = getBytes("akka.remote.client.message-frame-size").toInt + final val ConnectionTimeout = Duration(getMilliseconds("akka.remote.client.connection-timeout"), MILLISECONDS) + final val ReconnectionTimeWindow = Duration(getMilliseconds("akka.remote.client.reconnection-time-window"), MILLISECONDS) + final val ReadTimeout = Duration(getMilliseconds("akka.remote.client.read-timeout"), MILLISECONDS) + final val ReconnectDelay = Duration(getMilliseconds("akka.remote.client.reconnect-delay"), MILLISECONDS) + final val MessageFrameSize = getBytes("akka.remote.client.message-frame-size").toInt } class RemoteServerSettings { import scala.collection.JavaConverters._ - val MessageFrameSize = getBytes("akka.remote.server.message-frame-size").toInt - val SecureCookie: Option[String] = getString("akka.remote.secure-cookie") match { + final val MessageFrameSize = getBytes("akka.remote.server.message-frame-size").toInt + final val SecureCookie: Option[String] = getString("akka.remote.secure-cookie") match { case "" ⇒ None case cookie ⇒ Some(cookie) } - val RequireCookie = { + final val RequireCookie = { val requireCookie = getBoolean("akka.remote.server.require-cookie") if (requireCookie && SecureCookie.isEmpty) throw new ConfigurationException( "Configuration option 'akka.remote.server.require-cookie' is turned on but no secure cookie is defined in 'akka.remote.secure-cookie'.") requireCookie } - val UsePassiveConnections = getBoolean("akka.remote.use-passive-connections") + final val UsePassiveConnections = getBoolean("akka.remote.use-passive-connections") - val UntrustedMode = getBoolean("akka.remote.server.untrusted-mode") - val Hostname = getString("akka.remote.server.hostname") match { + final val UntrustedMode = getBoolean("akka.remote.server.untrusted-mode") + final val Hostname = getString("akka.remote.server.hostname") match { case "" ⇒ InetAddress.getLocalHost.getHostAddress case value ⇒ value } - val Port = getInt("akka.remote.server.port") match { + final val Port = getInt("akka.remote.server.port") match { case 0 ⇒ try { val s = new java.net.ServerSocket(0) try s.getLocalPort finally s.close() @@ -74,26 +76,26 @@ class RemoteSettings(val config: Config, val systemName: String) { case other ⇒ other } - val Backlog = getInt("akka.remote.server.backlog") + final val Backlog = getInt("akka.remote.server.backlog") - val ExecutionPoolKeepAlive = Duration(getMilliseconds("akka.remote.server.execution-pool-keepalive"), MILLISECONDS) + final val ExecutionPoolKeepAlive = Duration(getMilliseconds("akka.remote.server.execution-pool-keepalive"), MILLISECONDS) - val ExecutionPoolSize = getInt("akka.remote.server.execution-pool-size") match { + final val ExecutionPoolSize = getInt("akka.remote.server.execution-pool-size") match { case sz if sz < 1 ⇒ throw new IllegalArgumentException("akka.remote.server.execution-pool-size is less than 1") case sz ⇒ sz } - val MaxChannelMemorySize = getBytes("akka.remote.server.max-channel-memory-size") match { + final val MaxChannelMemorySize = getBytes("akka.remote.server.max-channel-memory-size") match { case sz if sz < 0 ⇒ throw new IllegalArgumentException("akka.remote.server.max-channel-memory-size is less than 0 bytes") case sz ⇒ sz } - val MaxTotalMemorySize = getBytes("akka.remote.server.max-total-memory-size") match { + final val MaxTotalMemorySize = getBytes("akka.remote.server.max-total-memory-size") match { case sz if sz < 0 ⇒ throw new IllegalArgumentException("akka.remote.server.max-total-memory-size is less than 0 bytes") case sz ⇒ sz } // TODO handle the system name right and move this to config file syntax - val URI = "akka://sys@" + Hostname + ":" + Port + final val URI = "akka://sys@" + Hostname + ":" + Port } } \ No newline at end of file diff --git a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala index 0034bf25f9..b0f24592de 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -60,7 +60,9 @@ abstract class RemoteClient private[akka] ( * Converts the message to the wireprotocol and sends the message across the wire */ def send(message: Any, senderOption: Option[ActorRef], recipient: ActorRef): Unit = if (isRunning) { - log.debug("Sending message {} from {} to {}", message, senderOption, recipient) + if (remoteSupport.remote.remoteSettings.LogSentMessages) + log.debug("Sending message {} from {} to {}", message, senderOption, recipient) + send((message, senderOption, recipient)) } else { val exception = new RemoteClientException("RemoteModule client is not running, make sure you have invoked 'RemoteClient.connect()' before using it.", remoteSupport, remoteAddress) diff --git a/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala index 13adbf31e8..32db14194a 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala @@ -26,6 +26,9 @@ class RemoteConfigSpec extends AkkaSpec("") { getBoolean("akka.remote.server.untrusted-mode") must equal(false) getInt("akka.remote.server.backlog") must equal(4096) + getBoolean("akka.remote.log-received-messages") must equal(false) + getBoolean("akka.remote.log-sent-messages") must equal(false) + getMilliseconds("akka.remote.server.execution-pool-keepalive") must equal(60 * 1000) getInt("akka.remote.server.execution-pool-size") must equal(4) From 184d15d81e4d20618c804c2d9475af87f4285fce Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sat, 21 Jan 2012 01:06:32 +0100 Subject: [PATCH 083/152] Switching to tail recursion and lowering numer of moving parts --- .../src/main/scala/akka/actor/ActorCell.scala | 1 + .../main/scala/akka/dispatch/Mailbox.scala | 35 ++++++------------- 2 files changed, 11 insertions(+), 25 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 0e813e5cd2..9eec626ecc 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -471,6 +471,7 @@ private[akka] class ActorCell( //Memory consistency is handled by the Mailbox (reading mailbox status then processing messages, then writing mailbox status final def invoke(messageHandle: Envelope) { + if (Mailbox.debug) println(self + " processing message " + messageHandle) try { currentMessage = messageHandle try { diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index dacf39593d..c81d7d8079 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -175,32 +175,17 @@ private[akka] abstract class Mailbox(val actor: ActorCell) extends MessageQueue * * @return true if the processing finished before the mailbox was empty, due to the throughput constraint */ - private final def processMailbox() { - if (shouldProcessMessage) { - var nextMessage = dequeue() - if (nextMessage ne null) { //If we have a message - if (dispatcher.isThroughputDefined) { //If we're using throughput, we need to do some book-keeping - var processedMessages = 0 - val deadlineNs = if (dispatcher.isThroughputDeadlineTimeDefined) System.nanoTime + dispatcher.throughputDeadlineTime.toNanos else 0 - do { - if (debug) println(actor.self + " processing message " + nextMessage) - actor invoke nextMessage - processAllSystemMessages() //After we're done, process all system messages + private final def processMailbox(): Unit = if (dispatcher.isThroughputDefined) process(dispatcher.throughput) else process(1) - nextMessage = if (shouldProcessMessage) { // If we aren't suspended, we need to make sure we're not overstepping our boundaries - processedMessages += 1 - if ((processedMessages >= dispatcher.throughput) || (dispatcher.isThroughputDeadlineTimeDefined && System.nanoTime >= deadlineNs)) // If we're throttled, break out - null //We reached our boundaries, abort - else dequeue //Dequeue the next message - } else null //Abort - } while (nextMessage ne null) - } else { //If we only run one message per process - actor invoke nextMessage //Just run it - processAllSystemMessages() //After we're done, process all system messages - } - } - } - } + @tailrec private final def process(left: Int, deadlineNs: Long = if (dispatcher.isThroughputDeadlineTimeDefined) System.nanoTime + dispatcher.throughputDeadlineTime.toNanos else 0l): Unit = + if ((shouldProcessMessage) && (left > 0 || (dispatcher.isThroughputDeadlineTimeDefined && System.nanoTime >= deadlineNs))) { + val next = dequeue() + if (next ne null) { + actor invoke next + processAllSystemMessages() + process(left - 1, deadlineNs) + } else () + } else () final def processAllSystemMessages() { var nextMessage = systemDrain() From bcfd70f6e0f6e78302401678d45adb7b22432103 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sat, 21 Jan 2012 01:09:06 +0100 Subject: [PATCH 084/152] Moving debug logging statement back into Mailbox.scala --- akka-actor/src/main/scala/akka/actor/ActorCell.scala | 1 - akka-actor/src/main/scala/akka/dispatch/Mailbox.scala | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 9eec626ecc..0e813e5cd2 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -471,7 +471,6 @@ private[akka] class ActorCell( //Memory consistency is handled by the Mailbox (reading mailbox status then processing messages, then writing mailbox status final def invoke(messageHandle: Envelope) { - if (Mailbox.debug) println(self + " processing message " + messageHandle) try { currentMessage = messageHandle try { diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index c81d7d8079..383db2f92e 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -181,6 +181,7 @@ private[akka] abstract class Mailbox(val actor: ActorCell) extends MessageQueue if ((shouldProcessMessage) && (left > 0 || (dispatcher.isThroughputDeadlineTimeDefined && System.nanoTime >= deadlineNs))) { val next = dequeue() if (next ne null) { + if (Mailbox.debug) println(actor.self + " processing message " + next) actor invoke next processAllSystemMessages() process(left - 1, deadlineNs) From 388ec810bf1cbef1bad3da2f8a3a0ac34c2ee213 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sat, 21 Jan 2012 01:13:55 +0100 Subject: [PATCH 085/152] Fixing inverted logic --- akka-actor/src/main/scala/akka/dispatch/Mailbox.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index 383db2f92e..abed403c44 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -178,7 +178,7 @@ private[akka] abstract class Mailbox(val actor: ActorCell) extends MessageQueue private final def processMailbox(): Unit = if (dispatcher.isThroughputDefined) process(dispatcher.throughput) else process(1) @tailrec private final def process(left: Int, deadlineNs: Long = if (dispatcher.isThroughputDeadlineTimeDefined) System.nanoTime + dispatcher.throughputDeadlineTime.toNanos else 0l): Unit = - if ((shouldProcessMessage) && (left > 0 || (dispatcher.isThroughputDeadlineTimeDefined && System.nanoTime >= deadlineNs))) { + if ((left > 0) && (shouldProcessMessage) && (!dispatcher.isThroughputDeadlineTimeDefined || System.nanoTime < deadlineNs)) { val next = dequeue() if (next ne null) { if (Mailbox.debug) println(actor.self + " processing message " + next) From a22d01f5954bb2987db49e17c0d2de41b814e9f7 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sat, 21 Jan 2012 01:15:00 +0100 Subject: [PATCH 086/152] Removing noise --- akka-actor/src/main/scala/akka/dispatch/Mailbox.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index abed403c44..195f77ee94 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -185,8 +185,8 @@ private[akka] abstract class Mailbox(val actor: ActorCell) extends MessageQueue actor invoke next processAllSystemMessages() process(left - 1, deadlineNs) - } else () - } else () + } + } final def processAllSystemMessages() { var nextMessage = systemDrain() From 42d33288158538cc61305e69b360bd24f86996f7 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sat, 21 Jan 2012 01:33:07 +0100 Subject: [PATCH 087/152] Moving tests after first invocation --- .../scala/akka/dispatch/AbstractDispatcher.scala | 2 -- .../src/main/scala/akka/dispatch/Mailbox.scala | 13 ++++++------- 2 files changed, 6 insertions(+), 9 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index 12c8c89d25..29be04fe40 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -282,8 +282,6 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext @inline protected[akka] final val isThroughputDeadlineTimeDefined = throughputDeadlineTime.toMillis > 0 - @inline - protected[akka] final val isThroughputDefined = throughput > 1 protected[akka] def executeTask(invocation: TaskInvocation) diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index 195f77ee94..e0bda0c41e 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -172,19 +172,18 @@ private[akka] abstract class Mailbox(val actor: ActorCell) extends MessageQueue /** * Process the messages in the mailbox - * - * @return true if the processing finished before the mailbox was empty, due to the throughput constraint */ - private final def processMailbox(): Unit = if (dispatcher.isThroughputDefined) process(dispatcher.throughput) else process(1) - - @tailrec private final def process(left: Int, deadlineNs: Long = if (dispatcher.isThroughputDeadlineTimeDefined) System.nanoTime + dispatcher.throughputDeadlineTime.toNanos else 0l): Unit = - if ((left > 0) && (shouldProcessMessage) && (!dispatcher.isThroughputDeadlineTimeDefined || System.nanoTime < deadlineNs)) { + @tailrec private final def processMailbox( + left: Int = java.lang.Math.min(dispatcher.throughput, 1), + deadlineNs: Long = if (dispatcher.isThroughputDeadlineTimeDefined) System.nanoTime + dispatcher.throughputDeadlineTime.toNanos else 0l): Unit = + if (shouldProcessMessage) { val next = dequeue() if (next ne null) { if (Mailbox.debug) println(actor.self + " processing message " + next) actor invoke next processAllSystemMessages() - process(left - 1, deadlineNs) + if ((left > 1) && (!dispatcher.isThroughputDeadlineTimeDefined || System.nanoTime < deadlineNs)) + processMailbox(left - 1, deadlineNs) } } From 6a4d7f547c9a951e737e4a899dc98398de53265b Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Sat, 21 Jan 2012 11:05:30 +0100 Subject: [PATCH 088/152] Improvement based on feedback. See #1702 --- .../akka/docs/extension/SettingsExtensionDocTestBase.java | 4 ++-- .../code/akka/docs/extension/SettingsExtensionDocSpec.scala | 3 ++- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTestBase.java b/akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTestBase.java index bbe2b2bcdc..b0bddba4b2 100644 --- a/akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTestBase.java +++ b/akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTestBase.java @@ -53,8 +53,8 @@ public class SettingsExtensionDocTestBase { //#extension-usage-actor public static class MyActor extends UntypedActor { - Connection connection = connect(Settings.instance.get(getContext().system()).DB_URI, - Settings.instance.get(getContext().system()).CIRCUIT_BREAKER_TIMEOUT); + final SettingsImpl settings = Settings.instance.get(getContext().system()); + Connection connection = connect(settings.DB_URI, settings.CIRCUIT_BREAKER_TIMEOUT); //#extension-usage-actor diff --git a/akka-docs/scala/code/akka/docs/extension/SettingsExtensionDocSpec.scala b/akka-docs/scala/code/akka/docs/extension/SettingsExtensionDocSpec.scala index 91e4f0ba1a..ed5ec66517 100644 --- a/akka-docs/scala/code/akka/docs/extension/SettingsExtensionDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/extension/SettingsExtensionDocSpec.scala @@ -51,7 +51,8 @@ object SettingsExtensionDocSpec { //#extension-usage-actor class MyActor extends Actor { - val connection = connect(Settings(context.system).DbUri, Settings(context.system).CircuitBreakerTimeout) + val settings = Settings(context.system) + val connection = connect(settings.DbUri, settings.CircuitBreakerTimeout) //#extension-usage-actor def receive = { From 61e0bd568b64e301ba3a01282f4e9b3036a056d7 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sat, 21 Jan 2012 13:04:45 +0100 Subject: [PATCH 089/152] Modifications after review --- akka-actor/src/main/scala/akka/dispatch/Mailbox.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index e0bda0c41e..4c180861af 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -174,15 +174,15 @@ private[akka] abstract class Mailbox(val actor: ActorCell) extends MessageQueue * Process the messages in the mailbox */ @tailrec private final def processMailbox( - left: Int = java.lang.Math.min(dispatcher.throughput, 1), - deadlineNs: Long = if (dispatcher.isThroughputDeadlineTimeDefined) System.nanoTime + dispatcher.throughputDeadlineTime.toNanos else 0l): Unit = + left: Int = java.lang.Math.max(dispatcher.throughput, 1), + deadlineNs: Long = if (dispatcher.isThroughputDeadlineTimeDefined) System.nanoTime + dispatcher.throughputDeadlineTime.toNanos else 0L): Unit = if (shouldProcessMessage) { val next = dequeue() if (next ne null) { if (Mailbox.debug) println(actor.self + " processing message " + next) actor invoke next processAllSystemMessages() - if ((left > 1) && (!dispatcher.isThroughputDeadlineTimeDefined || System.nanoTime < deadlineNs)) + if ((left > 1) && ((!dispatcher.isThroughputDeadlineTimeDefined) || (System.nanoTime - deadlineNs) < 0)) processMailbox(left - 1, deadlineNs) } } From e523d5418cadb7bdca67d2fc54b8105a238d35b0 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sat, 21 Jan 2012 13:12:30 +0100 Subject: [PATCH 090/152] Clarified the code, removing a negation that could be overlooked --- akka-actor/src/main/scala/akka/dispatch/Mailbox.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index 4c180861af..40d4650f0d 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -175,14 +175,14 @@ private[akka] abstract class Mailbox(val actor: ActorCell) extends MessageQueue */ @tailrec private final def processMailbox( left: Int = java.lang.Math.max(dispatcher.throughput, 1), - deadlineNs: Long = if (dispatcher.isThroughputDeadlineTimeDefined) System.nanoTime + dispatcher.throughputDeadlineTime.toNanos else 0L): Unit = + deadlineNs: Long = if (dispatcher.isThroughputDeadlineTimeDefined == true) System.nanoTime + dispatcher.throughputDeadlineTime.toNanos else 0L): Unit = if (shouldProcessMessage) { val next = dequeue() if (next ne null) { if (Mailbox.debug) println(actor.self + " processing message " + next) actor invoke next processAllSystemMessages() - if ((left > 1) && ((!dispatcher.isThroughputDeadlineTimeDefined) || (System.nanoTime - deadlineNs) < 0)) + if ((left > 1) && ((dispatcher.isThroughputDeadlineTimeDefined == false) || (System.nanoTime - deadlineNs) < 0)) processMailbox(left - 1, deadlineNs) } } From f0bc2c7435c8898b695996b067d754407ab01f56 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sat, 21 Jan 2012 13:29:16 +0100 Subject: [PATCH 091/152] Adding brackets around remote debug log parameters --- akka-remote/src/main/scala/akka/remote/Remote.scala | 2 +- .../src/main/scala/akka/remote/netty/NettyRemoteSupport.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/Remote.scala b/akka-remote/src/main/scala/akka/remote/Remote.scala index 04b1bf83a4..afaa14ec13 100644 --- a/akka-remote/src/main/scala/akka/remote/Remote.scala +++ b/akka-remote/src/main/scala/akka/remote/Remote.scala @@ -240,7 +240,7 @@ trait RemoteMarshallingOps { def receiveMessage(remoteMessage: RemoteMessage) { if (remote.remoteSettings.LogReceivedMessages) - log.debug("received message {}", remoteMessage) + log.debug("received message [{}]", remoteMessage) val remoteDaemon = remote.remoteDaemon diff --git a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala index b0f24592de..4e80c84fb5 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -61,7 +61,7 @@ abstract class RemoteClient private[akka] ( */ def send(message: Any, senderOption: Option[ActorRef], recipient: ActorRef): Unit = if (isRunning) { if (remoteSupport.remote.remoteSettings.LogSentMessages) - log.debug("Sending message {} from {} to {}", message, senderOption, recipient) + log.debug("Sending message [{}] from [{}] to [{}]", message, senderOption, recipient) send((message, senderOption, recipient)) } else { From 6cb1914b5c4e2ea8ea864b353b3bd8643c80899d Mon Sep 17 00:00:00 2001 From: viktorklang Date: Sun, 22 Jan 2012 23:11:09 +0100 Subject: [PATCH 092/152] Update akka-actor/src/main/scala/akka/dispatch/Future.scala --- akka-actor/src/main/scala/akka/dispatch/Future.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 505742fc56..7af41035e5 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -1,4 +1,3 @@ - /** * Copyright (C) 2009-2012 Typesafe Inc. */ @@ -269,6 +268,11 @@ object Future { } /** + * Signals that the current thread of execution will potentially engage + * in blocking calls after the call to this method, giving the system a + * chance to spawn new threads, reuse old threads or otherwise, to prevent + * starvation and/or unfairness. + * * Assures that any Future tasks initiated in the current thread will be * executed asynchronously, including any tasks currently queued to be * executed in the current thread. This is needed if the current task may From 651a426de03a63621b11271a9f482e28f7aca368 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 23 Jan 2012 11:21:02 +0100 Subject: [PATCH 093/152] weird star replacement --- akka-actor/src/main/scala/akka/dispatch/Future.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 7af41035e5..654b561b94 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -272,7 +272,7 @@ object Future { * in blocking calls after the call to this method, giving the system a * chance to spawn new threads, reuse old threads or otherwise, to prevent * starvation and/or unfairness. - * + * * Assures that any Future tasks initiated in the current thread will be * executed asynchronously, including any tasks currently queued to be * executed in the current thread. This is needed if the current task may From dd3a6ad4a55074ec64608f1dec5520ccbd3f0a68 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 23 Jan 2012 11:39:16 +0100 Subject: [PATCH 094/152] Unborking master, sorry about that. I blame my agent. --- akka-actor/src/main/scala/akka/dispatch/Future.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index e86a20ca4c..63dcbf6a35 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -321,7 +321,7 @@ object Future { val taskStack = Stack.empty[() ⇒ Unit] taskStack push task - _taskStack set Some(taskStack) + _taskStack set taskStack while (taskStack.nonEmpty) { val next = taskStack.pop() From 71d58c5338d50d1d798f545fcf98264b4db472c8 Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 23 Jan 2012 11:58:27 +0100 Subject: [PATCH 095/152] improve migration guide & helpers, plus some docs --- .../src/main/scala/akka/actor/OldActor.scala | 4 +- .../main/scala/akka/pattern/Patterns.scala | 11 ++- .../project/migration-guide-1.3.x-2.0.x.rst | 68 +++++++++++++++---- 3 files changed, 64 insertions(+), 19 deletions(-) diff --git a/akka-actor-migration/src/main/scala/akka/actor/OldActor.scala b/akka-actor-migration/src/main/scala/akka/actor/OldActor.scala index 0a9238209e..e923773bd8 100644 --- a/akka-actor-migration/src/main/scala/akka/actor/OldActor.scala +++ b/akka-actor-migration/src/main/scala/akka/actor/OldActor.scala @@ -10,6 +10,7 @@ import akka.dispatch.OldFuture import akka.util.Duration import java.util.concurrent.TimeUnit import java.net.InetSocketAddress +import akka.migration.AskableActorRef /** * Migration replacement for `object akka.actor.Actor`. @@ -54,7 +55,6 @@ object OldActor { @deprecated("OldActor.remote should not be used", "2.0") lazy val remote: OldRemoteSupport = new OldRemoteSupport - } @deprecated("use Actor", "2.0") @@ -66,6 +66,8 @@ abstract class OldActor extends Actor { implicit def actorRef2OldActorRef(actorRef: ActorRef) = new OldActorRef(actorRef) + implicit def askableActorRef(actorRef: ActorRef): AskableActorRef = new AskableActorRef(actorRef) + @deprecated("Use context.become instead", "2.0") def become(behavior: Receive, discardOld: Boolean = true) = context.become(behavior, discardOld) diff --git a/akka-actor/src/main/scala/akka/pattern/Patterns.scala b/akka-actor/src/main/scala/akka/pattern/Patterns.scala index 600efc367c..b1498ee2d0 100644 --- a/akka-actor/src/main/scala/akka/pattern/Patterns.scala +++ b/akka-actor/src/main/scala/akka/pattern/Patterns.scala @@ -112,13 +112,12 @@ object Patterns { * Recommended usage example: * * {{{ - * val f = ask(worker, request)(timeout) - * flow { - * EnrichedRequest(request, f()) - * } pipeTo nextActor + * final Future f = Patterns.ask(worker, request, timeout); + * // apply some transformation (i.e. enrich with request info) + * final Future transformed = f.map(new akka.japi.Function() { ... }); + * // send it on to the next stage + * Patterns.pipeTo(transformed, nextActor); * }}} - * - * [see [[akka.dispatch.Future]] for a description of `flow`] */ def pipeTo[T](future: Future[T], actorRef: ActorRef): Future[T] = akka.pattern.pipeTo(future, actorRef) diff --git a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst index 33752b8ce7..273d44c1d1 100644 --- a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst +++ b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst @@ -22,8 +22,8 @@ anything is able to run again. Therefore we provide a migration kit that makes it possible to do the migration changes in smaller steps. The migration kit only covers the most common usage of Akka. It is not intended -as a final solution. The whole migration kit is deprecated and will be removed in -Akka 2.1. +as a final solution. The whole migration kit is marked as deprecated and will +be removed in Akka 2.1. The migration kit is provided in separate jar files. Add the following dependency:: @@ -136,7 +136,8 @@ v1.3:: v2.0:: - system.shutdown() + system.shutdown() // from outside of this system + context.system.shutdown() // from inside any actor Documentation: @@ -149,7 +150,11 @@ Identifying Actors In v1.3 actors have ``uuid`` and ``id`` field. In v2.0 each actor has a unique logical ``path``. The ``ActorRegistry`` has been replaced by actor paths and lookup with -``actorFor`` in ``ActorRefProvider`` (``ActorSystem`` or ``ActorContext``). +``actorFor`` in ``ActorRefProvider`` (``ActorSystem`` or ``ActorContext``). It +is no longer possible to obtain references to all actors being implemented by a +certain class (the reason being that this property is not known yet when an +:class:`ActorRef` is created because instantiation of the actor itself is +asynchronous). v1.3:: @@ -170,7 +175,9 @@ Reply to messages ^^^^^^^^^^^^^^^^^ ``self.channel`` has been replaced with unified reply mechanism using ``sender`` (Scala) -or ``getSender()`` (Java). This works for both tell (!) and ask (?). +or ``getSender()`` (Java). This works for both tell (!) and ask (?). Sending to +an actor reference never throws an exception, hence :meth:`tryTell` and +:meth:`tryReply` are removed. v1.3:: @@ -200,6 +207,31 @@ reply to be received; it is independent of the timeout applied when awaiting completion of the :class:`Future`, however, the actor will complete the :class:`Future` with an :class:`AskTimeoutException` when it stops itself. +Also, since the ``ask`` feature is coupling futures and actors, it is no longer +offered on the :class:`ActorRef` itself, but instead as a use pattern to be +imported. While Scala’s implicit conversions enable transparent replacement, +Java code will have to be changed by more than just adding an import statement. + +v1.3:: + + actorRef ? message // Scala + actorRef.ask(message, timeout); // Java + +v2.0 (Scala):: + + import akka.pattern.ask + + actorRef ? message + ask(actorRef, message) // will use `akka.actor.timeout` or implicit Timeout + ask(actorRef, message)(timeout) + +v2.0 (Java):: + + import akka.pattern.Patterns; + + Patterns.ask(actorRef, message) // will use `akka.actor.timeout` + Patterns.ask(actorRef, message, timeout) + Documentation: * :ref:`actors-scala` @@ -325,7 +357,8 @@ v2.0:: import akka.event.Logging - val log = Logging(context.system, this) + val log = Logging(context.system, this) // will include system name in message source + val log = Logging(system.eventStream, this) // will not include system name log.error(exception, message) log.warning(message) log.info(message) @@ -501,17 +534,25 @@ Documentation: Spawn ^^^^^ -``spawn`` has been removed and can be implemented like this, if needed. Be careful to not +``spawn`` has been removed and should be replaced by creating a :class:`Future`. Be careful to not access any shared mutable state closed over by the body. -:: +Scala:: - def spawn(body: ⇒ Unit) { - system.actorOf(Props(ctx ⇒ { case "go" ⇒ try body finally ctx.stop(ctx.self) })) ! "go" - } + Future { doSomething() } // will be executed asynchronously + +Java:: + + Futures.future(new Callable() { + public String call() { + doSomething(); + } + }, executionContext); Documentation: + * :ref:`futures-scala` + * :ref:`futures-java` * :ref:`jmm` HotSwap @@ -521,7 +562,10 @@ In v2.0 ``become`` and ``unbecome`` metods are located in ``ActorContext``, i.e. The special ``HotSwap`` and ``RevertHotswap`` messages in v1.3 has been removed. Similar can be implemented with your own message and using ``context.become`` and ``context.unbecome`` -in the actor receiving the message. +in the actor receiving the message. The rationale is that being able to replace +any actor’s behavior generically is not a good idea because actor implementors +would have no way to defend against that; hence the change to lay it into the +hands of the actor itself. * :ref:`actors-scala` * :ref:`untyped-actors-java` From 66e0a7cf0bacbdc5b871c56643e93eeab2d1ae93 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 23 Jan 2012 13:49:19 +0100 Subject: [PATCH 096/152] Moved definition of fault handler from Props to overridable method supervisorStrategy in Actor. See #1711 * New trait SupervisorStrategy for TypedActors * Adjustments of docs * Updated tests --- .../ActorFireForgetRequestReplySpec.scala | 2 +- .../scala/akka/actor/ActorLifeCycleSpec.scala | 6 +- .../test/scala/akka/actor/ActorRefSpec.scala | 4 +- .../scala/akka/actor/DeathWatchSpec.scala | 18 +-- .../scala/akka/actor/FSMTransitionSpec.scala | 3 +- .../akka/actor/RestartStrategySpec.scala | 11 +- .../test/scala/akka/actor/SchedulerSpec.scala | 2 +- .../test/scala/akka/actor/Supervisor.scala | 7 +- .../akka/actor/SupervisorHierarchySpec.scala | 19 ++- .../scala/akka/actor/SupervisorMiscSpec.scala | 2 +- .../scala/akka/actor/SupervisorSpec.scala | 20 +-- .../scala/akka/actor/SupervisorTreeSpec.scala | 3 +- .../test/scala/akka/actor/Ticket669Spec.scala | 4 +- .../scala/akka/actor/TypedActorSpec.scala | 11 +- .../scala/akka/event/LoggingReceiveSpec.scala | 3 +- .../src/main/scala/akka/actor/Actor.scala | 14 ++ .../src/main/scala/akka/actor/ActorCell.scala | 10 +- .../scala/akka/actor/ActorRefProvider.scala | 20 +-- .../main/scala/akka/actor/FaultHandling.scala | 10 ++ .../src/main/scala/akka/actor/Props.scala | 28 +--- .../main/scala/akka/actor/TypedActor.scala | 27 ++-- .../main/scala/akka/actor/UntypedActor.scala | 26 ++++ .../src/main/scala/akka/dispatch/Future.scala | 2 +- akka-docs/general/actor-systems.rst | 134 +++++++++--------- .../docs/actor/FaultHandlingTestBase.java | 105 +++++++++----- akka-docs/java/fault-tolerance.rst | 10 +- .../docs/actor/FaultHandlingDocSpec.scala | 41 ++++-- akka-docs/scala/fault-tolerance.rst | 8 +- .../scala/akka/testkit/TestActorRefSpec.scala | 4 +- 29 files changed, 331 insertions(+), 223 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala index 13ccddefe2..9fa3ef2709 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala @@ -80,7 +80,7 @@ class ActorFireForgetRequestReplySpec extends AkkaSpec with BeforeAndAfterEach w "should shutdown crashed temporary actor" in { filterEvents(EventFilter[Exception]("Expected exception")) { - val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(0)))) + val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), Some(0))))) val actor = Await.result((supervisor ? Props[CrashingActor]).mapTo[ActorRef], timeout.duration) actor.isTerminated must be(false) actor ! "Die" diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala index b203ff256f..5fb9187cf7 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala @@ -35,7 +35,7 @@ class ActorLifeCycleSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitS "invoke preRestart, preStart, postRestart when using OneForOneStrategy" in { filterException[ActorKilledException] { val id = newUuid().toString - val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(3)))) + val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), Some(3))))) val gen = new AtomicInteger(0) val restarterProps = Props(new LifeCycleTestActor(testActor, id, gen) { override def preRestart(reason: Throwable, message: Option[Any]) { report("preRestart") } @@ -69,7 +69,7 @@ class ActorLifeCycleSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitS "default for preRestart and postRestart is to call postStop and preStart respectively" in { filterException[ActorKilledException] { val id = newUuid().toString - val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(3)))) + val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), Some(3))))) val gen = new AtomicInteger(0) val restarterProps = Props(new LifeCycleTestActor(testActor, id, gen)) val restarter = Await.result((supervisor ? restarterProps).mapTo[ActorRef], timeout.duration) @@ -99,7 +99,7 @@ class ActorLifeCycleSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitS "not invoke preRestart and postRestart when never restarted using OneForOneStrategy" in { val id = newUuid().toString - val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(3)))) + val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), Some(3))))) val gen = new AtomicInteger(0) val props = Props(new LifeCycleTestActor(testActor, id, gen)) val a = Await.result((supervisor ? props).mapTo[ActorRef], timeout.duration) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala index 63f8ca242e..b4ab7f066d 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala @@ -374,6 +374,8 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout { val boss = system.actorOf(Props(new Actor { + override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), 2, 1000) + val ref = context.actorOf( Props(new Actor { def receive = { case _ ⇒ } @@ -382,7 +384,7 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout { })) protected def receive = { case "sendKill" ⇒ ref ! Kill } - }).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 2, 1000))) + })) boss ! "sendKill" Await.ready(latch, 5 seconds) diff --git a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala index cd6dc58129..78fd2a6780 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala @@ -23,7 +23,7 @@ trait DeathWatchSpec { this: AkkaSpec with ImplicitSender with DefaultTimeout import DeathWatchSpec._ - lazy val supervisor = system.actorOf(Props[Supervisor], "watchers") + lazy val supervisor = system.actorOf(Props(new Supervisor(FaultHandlingStrategy.defaultFaultHandler)), "watchers") def startWatching(target: ActorRef) = Await.result((supervisor ? props(target, testActor)).mapTo[ActorRef], 3 seconds) @@ -94,7 +94,7 @@ trait DeathWatchSpec { this: AkkaSpec with ImplicitSender with DefaultTimeout "notify with a Terminated message once when an Actor is stopped but not when restarted" in { filterException[ActorKilledException] { - val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(2)))) + val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), Some(2))))) val terminalProps = Props(context ⇒ { case x ⇒ context.sender ! x }) val terminal = Await.result((supervisor ? terminalProps).mapTo[ActorRef], timeout.duration) @@ -115,13 +115,13 @@ trait DeathWatchSpec { this: AkkaSpec with ImplicitSender with DefaultTimeout "fail a monitor which does not handle Terminated()" in { filterEvents(EventFilter[ActorKilledException](), EventFilter[DeathPactException]()) { case class FF(fail: Failed) - val supervisor = system.actorOf(Props[Supervisor] - .withFaultHandler(new OneForOneStrategy(FaultHandlingStrategy.makeDecider(List(classOf[Exception])), Some(0)) { - override def handleFailure(context: ActorContext, child: ActorRef, cause: Throwable, stats: ChildRestartStats, children: Iterable[ChildRestartStats]) = { - testActor.tell(FF(Failed(cause)), child) - super.handleFailure(context, child, cause, stats, children) - } - })) + val strategy = new OneForOneStrategy(FaultHandlingStrategy.makeDecider(List(classOf[Exception])), Some(0)) { + override def handleFailure(context: ActorContext, child: ActorRef, cause: Throwable, stats: ChildRestartStats, children: Iterable[ChildRestartStats]) = { + testActor.tell(FF(Failed(cause)), child) + super.handleFailure(context, child, cause, stats, children) + } + } + val supervisor = system.actorOf(Props(new Supervisor(strategy))) val failed = Await.result((supervisor ? Props.empty).mapTo[ActorRef], timeout.duration) val brother = Await.result((supervisor ? Props(new Actor { diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala index cc63b76704..36258883e9 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala @@ -72,8 +72,9 @@ class FSMTransitionSpec extends AkkaSpec with ImplicitSender { val fsm = system.actorOf(Props(new MyFSM(testActor))) val sup = system.actorOf(Props(new Actor { context.watch(fsm) + override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), None, None) def receive = { case _ ⇒ } - }).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, None))) + })) within(300 millis) { fsm ! SubscribeTransitionCallBack(forward) diff --git a/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala index e2fd22a030..4b7853af95 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala @@ -28,7 +28,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout { "A RestartStrategy" must { "ensure that slave stays dead after max restarts within time range" in { - val boss = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 2, 1000))) + val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Throwable]), 2, 1000)))) val restartLatch = new TestLatch val secondRestartLatch = new TestLatch @@ -74,7 +74,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout { } "ensure that slave is immortal without max restarts and time range" in { - val boss = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, None))) + val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Throwable]), None, None)))) val countDownLatch = new TestLatch(100) @@ -96,7 +96,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout { } "ensure that slave restarts after number of crashes not within time range" in { - val boss = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 2, 500))) + val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Throwable]), 2, 500)))) val restartLatch = new TestLatch val secondRestartLatch = new TestLatch @@ -153,7 +153,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout { } "ensure that slave is not restarted after max retries" in { - val boss = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), Some(2), None))) + val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Throwable]), Some(2), None)))) val restartLatch = new TestLatch val secondRestartLatch = new TestLatch @@ -208,11 +208,12 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout { val countDownLatch = new TestLatch(2) val boss = system.actorOf(Props(new Actor { + override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), None, Some(1000)) def receive = { case p: Props ⇒ sender ! context.watch(context.actorOf(p)) case t: Terminated ⇒ maxNoOfRestartsLatch.open() } - }).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, Some(1000)))) + })) val slaveProps = Props(new Actor { diff --git a/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala index 6126911162..3a7e91a47d 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala @@ -133,7 +133,7 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout val restartLatch = new TestLatch val pingLatch = new TestLatch(6) - val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), 3, 1000))) + val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), 3, 1000)))) val props = Props(new Actor { def receive = { case Ping ⇒ pingLatch.countDown() diff --git a/akka-actor-tests/src/test/scala/akka/actor/Supervisor.scala b/akka-actor-tests/src/test/scala/akka/actor/Supervisor.scala index 0bf8183137..d04990c375 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/Supervisor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/Supervisor.scala @@ -3,7 +3,12 @@ */ package akka.actor -class Supervisor extends Actor { +/** + * For testing Supervisor behavior, normally you don't supply the strategy + * from the outside like this. + */ +class Supervisor(override val supervisorStrategy: FaultHandlingStrategy) extends Actor { + def receive = { case x: Props ⇒ sender ! context.actorOf(x) } diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala index 576c328bc7..2fc5996b54 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala @@ -12,7 +12,12 @@ import akka.dispatch.Await object SupervisorHierarchySpec { class FireWorkerException(msg: String) extends Exception(msg) - class CountDownActor(countDown: CountDownLatch) extends Actor { + /** + * For testing Supervisor behavior, normally you don't supply the strategy + * from the outside like this. + */ + class CountDownActor(countDown: CountDownLatch, override val supervisorStrategy: FaultHandlingStrategy) extends Actor { + protected def receive = { case p: Props ⇒ sender ! context.actorOf(p) } @@ -33,12 +38,12 @@ class SupervisorHierarchySpec extends AkkaSpec with DefaultTimeout { "restart manager and workers in AllForOne" in { val countDown = new CountDownLatch(4) - val boss = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), None, None))) + val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), None, None)))) - val managerProps = Props(new CountDownActor(countDown)).withFaultHandler(AllForOneStrategy(List(), None, None)) + val managerProps = Props(new CountDownActor(countDown, AllForOneStrategy(List(), None, None))) val manager = Await.result((boss ? managerProps).mapTo[ActorRef], timeout.duration) - val workerProps = Props(new CountDownActor(countDown)) + val workerProps = Props(new CountDownActor(countDown, FaultHandlingStrategy.defaultFaultHandler)) val workerOne, workerTwo, workerThree = Await.result((manager ? workerProps).mapTo[ActorRef], timeout.duration) filterException[ActorKilledException] { @@ -55,13 +60,15 @@ class SupervisorHierarchySpec extends AkkaSpec with DefaultTimeout { val countDownMessages = new CountDownLatch(1) val countDownMax = new CountDownLatch(1) val boss = system.actorOf(Props(new Actor { - val crasher = context.watch(context.actorOf(Props(new CountDownActor(countDownMessages)))) + override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), 1, 5000) + + val crasher = context.watch(context.actorOf(Props(new CountDownActor(countDownMessages, FaultHandlingStrategy.defaultFaultHandler)))) protected def receive = { case "killCrasher" ⇒ crasher ! Kill case Terminated(_) ⇒ countDownMax.countDown() } - }).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 1, 5000))) + })) filterException[ActorKilledException] { boss ! "killCrasher" diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala index e269456e9b..6e63a31d9a 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala @@ -28,7 +28,7 @@ class SupervisorMiscSpec extends AkkaSpec(SupervisorMiscSpec.config) with Defaul filterEvents(EventFilter[Exception]("Kill")) { val countDownLatch = new CountDownLatch(4) - val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 3, 5000))) + val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), 3, 5000)))) val workerProps = Props(new Actor { override def postRestart(cause: Throwable) { countDownLatch.countDown() } diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala index 30b1a24493..4a0f8e997f 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala @@ -53,6 +53,8 @@ object SupervisorSpec { var s: ActorRef = _ + override val supervisorStrategy = OneForOneStrategy(List(classOf[Exception]), Some(0)) + def receive = { case Die ⇒ temp forward Die case Terminated(`temp`) ⇒ sendTo ! "terminated" @@ -75,45 +77,45 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende private def child(supervisor: ActorRef, props: Props): ActorRef = Await.result((supervisor ? props).mapTo[ActorRef], timeout.duration) def temporaryActorAllForOne = { - val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), Some(0)))) + val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), Some(0))))) val temporaryActor = child(supervisor, Props(new PingPongActor(testActor))) (temporaryActor, supervisor) } def singleActorAllForOne = { - val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) + val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis)))) val pingpong = child(supervisor, Props(new PingPongActor(testActor))) (pingpong, supervisor) } def singleActorOneForOne = { - val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) + val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis)))) val pingpong = child(supervisor, Props(new PingPongActor(testActor))) (pingpong, supervisor) } def multipleActorsAllForOne = { - val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) + val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis)))) val pingpong1, pingpong2, pingpong3 = child(supervisor, Props(new PingPongActor(testActor))) (pingpong1, pingpong2, pingpong3, supervisor) } def multipleActorsOneForOne = { - val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) + val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis)))) val pingpong1, pingpong2, pingpong3 = child(supervisor, Props(new PingPongActor(testActor))) (pingpong1, pingpong2, pingpong3, supervisor) } def nestedSupervisorsAllForOne = { - val topSupervisor = system.actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) + val topSupervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis)))) val pingpong1 = child(topSupervisor, Props(new PingPongActor(testActor))) - val middleSupervisor = child(topSupervisor, Props[Supervisor].withFaultHandler(AllForOneStrategy(Nil, 3, TimeoutMillis))) + val middleSupervisor = child(topSupervisor, Props(new Supervisor(AllForOneStrategy(Nil, 3, TimeoutMillis)))) val pingpong2, pingpong3 = child(middleSupervisor, Props(new PingPongActor(testActor))) (pingpong1, pingpong2, pingpong3, topSupervisor) @@ -141,7 +143,7 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende "A supervisor" must { "not restart child more times than permitted" in { - val master = system.actorOf(Props(new Master(testActor)).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(0)))) + val master = system.actorOf(Props(new Master(testActor))) master ! Die expectMsg(3 seconds, "terminated") @@ -277,7 +279,7 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende "must attempt restart when exception during restart" in { val inits = new AtomicInteger(0) - val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(classOf[Exception] :: Nil, 3, 10000))) + val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(classOf[Exception] :: Nil, 3, 10000)))) val dyingProps = Props(new Actor { inits.incrementAndGet diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala index 27ecec4863..ea97415e72 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala @@ -22,11 +22,12 @@ class SupervisorTreeSpec extends AkkaSpec with ImplicitSender with DefaultTimeou EventFilter[ActorKilledException](occurrences = 1) intercept { within(5 seconds) { val p = Props(new Actor { + override val supervisorStrategy = OneForOneStrategy(List(classOf[Exception]), 3, 1000) def receive = { case p: Props ⇒ sender ! context.actorOf(p) } override def preRestart(cause: Throwable, msg: Option[Any]) { testActor ! self.path } - }).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 3, 1000)) + }) val headActor = system.actorOf(p) val middleActor = Await.result((headActor ? p).mapTo[ActorRef], timeout.duration) val lastActor = Await.result((middleActor ? p).mapTo[ActorRef], timeout.duration) diff --git a/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala b/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala index 73f3b416e9..cc4e7b2f74 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala @@ -24,7 +24,7 @@ class Ticket669Spec extends AkkaSpec with BeforeAndAfterAll with ImplicitSender "A supervised actor with lifecycle PERMANENT" should { "be able to reply on failure during preRestart" in { filterEvents(EventFilter[Exception]("test", occurrences = 1)) { - val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), 5, 10000))) + val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), 5, 10000)))) val supervised = Await.result((supervisor ? Props[Supervised]).mapTo[ActorRef], timeout.duration) supervised.!("test")(testActor) @@ -35,7 +35,7 @@ class Ticket669Spec extends AkkaSpec with BeforeAndAfterAll with ImplicitSender "be able to reply on failure during postStop" in { filterEvents(EventFilter[Exception]("test", occurrences = 1)) { - val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), Some(0), None))) + val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), Some(0), None)))) val supervised = Await.result((supervisor ? Props[Supervised]).mapTo[ActorRef], timeout.duration) supervised.!("test")(testActor) diff --git a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala index 17bbd948c3..6ef7608ce2 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala @@ -298,10 +298,13 @@ class TypedActorSpec extends AkkaSpec(TypedActorSpec.config) "be able to handle exceptions when calling methods" in { filterEvents(EventFilter[IllegalStateException]("expected")) { - val boss = system.actorOf(Props(context ⇒ { - case p: TypedProps[_] ⇒ context.sender ! TypedActor(context).typedActorOf(p) - }).withFaultHandler(OneForOneStrategy { - case e: IllegalStateException if e.getMessage == "expected" ⇒ FaultHandlingStrategy.Resume + val boss = system.actorOf(Props(new Actor { + override val supervisorStrategy = OneForOneStrategy { + case e: IllegalStateException if e.getMessage == "expected" ⇒ FaultHandlingStrategy.Resume + } + def receive = { + case p: TypedProps[_] ⇒ context.sender ! TypedActor(context).typedActorOf(p) + } })) val t = Await.result((boss ? TypedProps[Bar](classOf[Foo], classOf[Bar]).withTimeout(2 seconds)).mapTo[Foo], timeout.duration) diff --git a/akka-actor-tests/src/test/scala/akka/event/LoggingReceiveSpec.scala b/akka-actor-tests/src/test/scala/akka/event/LoggingReceiveSpec.scala index 7e756657f5..cf620bf0fc 100644 --- a/akka-actor-tests/src/test/scala/akka/event/LoggingReceiveSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/event/LoggingReceiveSpec.scala @@ -15,6 +15,7 @@ import akka.actor._ object LoggingReceiveSpec { class TestLogActor extends Actor { + override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), 5, 5000) def receive = { case _ ⇒ } } } @@ -149,7 +150,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd within(3 seconds) { val lifecycleGuardian = appLifecycle.asInstanceOf[ActorSystemImpl].guardian val lname = lifecycleGuardian.path.toString - val supervisor = TestActorRef[TestLogActor](Props[TestLogActor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 5, 5000))) + val supervisor = TestActorRef[TestLogActor](Props[TestLogActor]) val sname = supervisor.path.toString val sclass = classOf[TestLogActor] diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index 2cdd41c36e..f48b367e56 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -141,6 +141,14 @@ object Actor { * * {{{ * class ExampleActor extends Actor { + * + * override val supervisorStrategy = OneForOneStrategy({ + * case _: ArithmeticException ⇒ Resume + * case _: NullPointerException ⇒ Restart + * case _: IllegalArgumentException ⇒ Stop + * case _: Exception ⇒ Escalate + * }: Decider, maxNrOfRetries = Some(10), withinTimeRange = Some(60000)) + * * def receive = { * // directly calculated reply * case Request(r) => sender ! calculate(r) @@ -224,6 +232,12 @@ trait Actor { */ protected def receive: Receive + /** + * User overridable definition the strategy to use for supervising + * child actors. + */ + def supervisorStrategy(): FaultHandlingStrategy = FaultHandlingStrategy.defaultFaultHandler + /** * User overridable callback. *

diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 0e813e5cd2..5aaf4ae8d5 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -395,7 +395,7 @@ private[akka] class ActorCell( dispatcher.resume(this) //FIXME should this be moved down? - props.faultHandler.handleSupervisorRestarted(cause, self, children) + actor.supervisorStrategy.handleSupervisorRestarted(cause, self, children) } catch { // TODO catching all and continue isn't good for OOME, ticket #1418 case e ⇒ try { @@ -491,11 +491,11 @@ private[akka] class ActorCell( // make sure that InterruptedException does not leave this thread if (e.isInstanceOf[InterruptedException]) { val ex = ActorInterruptedException(e) - props.faultHandler.handleSupervisorFailing(self, children) + actor.supervisorStrategy.handleSupervisorFailing(self, children) parent.tell(Failed(ex), self) throw e //Re-throw InterruptedExceptions as expected } else { - props.faultHandler.handleSupervisorFailing(self, children) + actor.supervisorStrategy.handleSupervisorFailing(self, children) parent.tell(Failed(e), self) } } finally { @@ -569,7 +569,7 @@ private[akka] class ActorCell( } final def handleFailure(child: ActorRef, cause: Throwable): Unit = childrenRefs.get(child.path.name) match { - case Some(stats) if stats.child == child ⇒ if (!props.faultHandler.handleFailure(this, child, cause, stats, childrenRefs.values)) throw cause + case Some(stats) if stats.child == child ⇒ if (!actor.supervisorStrategy.handleFailure(this, child, cause, stats, childrenRefs.values)) throw cause case Some(stats) ⇒ system.eventStream.publish(Warning(self.path.toString, clazz(actor), "dropping Failed(" + cause + ") from unknown child " + child + " matching names but not the same, was: " + stats.child)) case None ⇒ system.eventStream.publish(Warning(self.path.toString, clazz(actor), "dropping Failed(" + cause + ") from unknown child " + child)) } @@ -577,7 +577,7 @@ private[akka] class ActorCell( final def handleChildTerminated(child: ActorRef): Unit = { if (childrenRefs contains child.path.name) { childrenRefs -= child.path.name - props.faultHandler.handleChildTerminated(this, child, children) + actor.supervisorStrategy.handleChildTerminated(this, child, children) if (stopping && childrenRefs.isEmpty) doTerminate() } else system.locker ! ChildTerminated(child) } diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 5073725fa6..0da5c13e1d 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -334,6 +334,16 @@ class LocalActorRefProvider( * exceptions which might have occurred. */ private class Guardian extends Actor { + + override val supervisorStrategy = { + import akka.actor.FaultHandlingStrategy._ + OneForOneStrategy { + case _: ActorKilledException ⇒ Stop + case _: ActorInitializationException ⇒ Stop + case _: Exception ⇒ Restart + } + } + def receive = { case Terminated(_) ⇒ context.stop(self) case CreateChild(child, name) ⇒ sender ! (try context.actorOf(child, name) catch { case e: Exception ⇒ e }) @@ -366,15 +376,7 @@ class LocalActorRefProvider( override def preRestart(cause: Throwable, msg: Option[Any]) {} } - private val guardianFaultHandlingStrategy = { - import akka.actor.FaultHandlingStrategy._ - OneForOneStrategy { - case _: ActorKilledException ⇒ Stop - case _: ActorInitializationException ⇒ Stop - case _: Exception ⇒ Restart - } - } - private val guardianProps = Props(new Guardian).withFaultHandler(guardianFaultHandlingStrategy) + private val guardianProps = Props(new Guardian) /* * The problem is that ActorRefs need a reference to the ActorSystem to diff --git a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala index 8dc816f8ad..02c94a34bc 100644 --- a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala @@ -95,6 +95,16 @@ object FaultHandlingStrategy { */ def escalate = Escalate + final val defaultFaultHandler: FaultHandlingStrategy = { + def defaultDecider: Decider = { + case _: ActorInitializationException ⇒ Stop + case _: ActorKilledException ⇒ Stop + case _: Exception ⇒ Restart + case _ ⇒ Escalate + } + OneForOneStrategy(defaultDecider, None, None) + } + type Decider = PartialFunction[Throwable, Action] type JDecider = akka.japi.Function[Throwable, Action] type CauseAction = (Class[_ <: Throwable], Action) diff --git a/akka-actor/src/main/scala/akka/actor/Props.scala b/akka-actor/src/main/scala/akka/actor/Props.scala index 8cce3f35e5..c0b84d0017 100644 --- a/akka-actor/src/main/scala/akka/actor/Props.scala +++ b/akka-actor/src/main/scala/akka/actor/Props.scala @@ -18,19 +18,11 @@ import akka.routing._ * Used when creating new actors through; ActorSystem.actorOf and ActorContext.actorOf. */ object Props { - import FaultHandlingStrategy._ final val defaultCreator: () ⇒ Actor = () ⇒ throw new UnsupportedOperationException("No actor creator specified!") - final val defaultDecider: Decider = { - case _: ActorInitializationException ⇒ Stop - case _: ActorKilledException ⇒ Stop - case _: Exception ⇒ Restart - case _ ⇒ Escalate - } final val defaultRoutedProps: RouterConfig = NoRouter - final val defaultFaultHandler: FaultHandlingStrategy = OneForOneStrategy(defaultDecider, None, None) final val noHotSwap: Stack[Actor.Receive] = Stack.empty final val empty = new Props(() ⇒ new Actor { def receive = Actor.emptyBehavior }) @@ -79,8 +71,6 @@ object Props { def apply(behavior: ActorContext ⇒ Actor.Receive): Props = apply(new Actor { def receive = behavior(context) }) - def apply(faultHandler: FaultHandlingStrategy): Props = - apply(new Actor { def receive = { case _ ⇒ } }).withFaultHandler(faultHandler) } /** @@ -94,14 +84,10 @@ object Props { * val props = Props( * creator = .., * dispatcher = .., - * faultHandler = .., * routerConfig = .. * ) * val props = Props().withCreator(new MyActor) * val props = Props[MyActor].withRouter(RoundRobinRouter(..)) - * val props = Props[MyActor].withFaultHandler(OneForOneStrategy { - * case e: IllegalStateException ⇒ Resume - * }) * }}} * * Examples on Java API: @@ -114,14 +100,12 @@ object Props { * } * }); * Props props = new Props().withCreator(new UntypedActorFactory() { ... }); - * Props props = new Props(MyActor.class).withFaultHandler(new OneForOneStrategy(...)); * Props props = new Props(MyActor.class).withRouter(new RoundRobinRouter(..)); * }}} */ case class Props( creator: () ⇒ Actor = Props.defaultCreator, dispatcher: String = Dispatchers.DefaultDispatcherId, - faultHandler: FaultHandlingStrategy = Props.defaultFaultHandler, routerConfig: RouterConfig = Props.defaultRoutedProps) { /** @@ -129,16 +113,14 @@ case class Props( */ def this() = this( creator = Props.defaultCreator, - dispatcher = Dispatchers.DefaultDispatcherId, - faultHandler = Props.defaultFaultHandler) + dispatcher = Dispatchers.DefaultDispatcherId) /** * Java API. */ def this(factory: UntypedActorFactory) = this( creator = () ⇒ factory.create(), - dispatcher = Dispatchers.DefaultDispatcherId, - faultHandler = Props.defaultFaultHandler) + dispatcher = Dispatchers.DefaultDispatcherId) /** * Java API. @@ -146,7 +128,6 @@ case class Props( def this(actorClass: Class[_ <: Actor]) = this( creator = () ⇒ actorClass.newInstance, dispatcher = Dispatchers.DefaultDispatcherId, - faultHandler = Props.defaultFaultHandler, routerConfig = Props.defaultRoutedProps) /** @@ -175,11 +156,6 @@ case class Props( */ def withDispatcher(d: String) = copy(dispatcher = d) - /** - * Returns a new Props with the specified faulthandler set. - */ - def withFaultHandler(f: FaultHandlingStrategy) = copy(faultHandler = f) - /** * Returns a new Props with the specified router config set. */ diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index 15397ab966..9f81f4c754 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -218,6 +218,11 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi TypedActor.currentContext set null } + override def supervisorStrategy: FaultHandlingStrategy = me match { + case l: SupervisorStrategy ⇒ l.supervisorStrategy + case _ ⇒ super.supervisorStrategy + } + override def preStart(): Unit = me match { case l: PreStart ⇒ l.preStart() case _ ⇒ super.preStart() @@ -275,6 +280,17 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi } } + /** + * Mix this into your TypedActor to be able to define supervisor strategy + */ + trait SupervisorStrategy { + /** + * User overridable definition the strategy to use for supervising + * child actors. + */ + def supervisorStrategy: FaultHandlingStrategy = FaultHandlingStrategy.defaultFaultHandler + } + /** * Mix this into your TypedActor to be able to hook into its lifecycle */ @@ -355,7 +371,6 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi object TypedProps { val defaultDispatcherId: String = Dispatchers.DefaultDispatcherId - val defaultFaultHandler: FaultHandlingStrategy = akka.actor.Props.defaultFaultHandler val defaultTimeout: Option[Timeout] = None val defaultLoader: Option[ClassLoader] = None @@ -415,7 +430,6 @@ case class TypedProps[T <: AnyRef] protected[TypedProps] ( interfaces: Seq[Class[_]], creator: () ⇒ T, dispatcher: String = TypedProps.defaultDispatcherId, - faultHandler: FaultHandlingStrategy = TypedProps.defaultFaultHandler, timeout: Option[Timeout] = TypedProps.defaultTimeout, loader: Option[ClassLoader] = TypedProps.defaultLoader) { @@ -458,11 +472,6 @@ case class TypedProps[T <: AnyRef] protected[TypedProps] ( */ def withDispatcher(d: String) = copy(dispatcher = d) - /** - * Returns a new Props with the specified faulthandler set. - */ - def withFaultHandler(f: FaultHandlingStrategy) = copy(faultHandler = f) - /** * @returns a new Props that will use the specified ClassLoader to create its proxy class in * If loader is null, it will use the bootstrap classloader. @@ -512,8 +521,8 @@ case class TypedProps[T <: AnyRef] protected[TypedProps] ( import akka.actor.{ Props ⇒ ActorProps } def actorProps(): ActorProps = - if (dispatcher == ActorProps().dispatcher && faultHandler == ActorProps().faultHandler) ActorProps() - else ActorProps(dispatcher = dispatcher, faultHandler = faultHandler) + if (dispatcher == ActorProps().dispatcher) ActorProps() + else ActorProps(dispatcher = dispatcher) } case class ContextualTypedActorFactory(typedActor: TypedActorExtension, actorFactory: ActorContext) extends TypedActorFactory { diff --git a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala index 76af2cfb4e..69cb096e26 100644 --- a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala @@ -37,6 +37,26 @@ import akka.dispatch.{ MessageDispatcher, Promise } * } * } * + * private static FaultHandlingStrategy strategy = new OneForOneStrategy(new Function() { + * @Override + * public Action apply(Throwable t) { + * if (t instanceof ArithmeticException) { + * return resume(); + * } else if (t instanceof NullPointerException) { + * return restart(); + * } else if (t instanceof IllegalArgumentException) { + * return stop(); + * } else { + * return escalate(); + * } + * } + * }, 10, 60000); + * + * @Override + * public FaultHandlingStrategy supervisorStrategy() { + * return strategy; + * } + * * public void onReceive(Object message) throws Exception { * if (message instanceof String) { * String msg = (String)message; @@ -92,6 +112,12 @@ abstract class UntypedActor extends Actor { */ def getSender(): ActorRef = sender + /** + * User overridable definition the strategy to use for supervising + * child actors. + */ + override def supervisorStrategy(): FaultHandlingStrategy = super.supervisorStrategy() + /** * User overridable callback. *

diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 7af41035e5..654b561b94 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -272,7 +272,7 @@ object Future { * in blocking calls after the call to this method, giving the system a * chance to spawn new threads, reuse old threads or otherwise, to prevent * starvation and/or unfairness. - * + * * Assures that any Future tasks initiated in the current thread will be * executed asynchronously, including any tasks currently queued to be * executed in the current thread. This is needed if the current task may diff --git a/akka-docs/general/actor-systems.rst b/akka-docs/general/actor-systems.rst index 6dfb5cf731..33b5a95fb8 100644 --- a/akka-docs/general/actor-systems.rst +++ b/akka-docs/general/actor-systems.rst @@ -3,90 +3,90 @@ Actor Systems ============= -Actors are objects which encapsulate state and behavior, they communicate -exclusively by exchanging messages which are placed into the recipient’s -mailbox. In a sense, actors are the most strigent form of object-oriented -programming, but it serves better to view them as persons: while modeling a -solution with actors, envision a group of people and assign sub-tasks to them, -arrange their functions into an organizational structure and think about how to -escalate failure (all with the benefit of not actually dealing with people, -which means that we need not concern ourselves with their emotional state or -moral issues). The result can then serve as a mental scaffolding for building +Actors are objects which encapsulate state and behavior, they communicate +exclusively by exchanging messages which are placed into the recipient’s +mailbox. In a sense, actors are the most strigent form of object-oriented +programming, but it serves better to view them as persons: while modeling a +solution with actors, envision a group of people and assign sub-tasks to them, +arrange their functions into an organizational structure and think about how to +escalate failure (all with the benefit of not actually dealing with people, +which means that we need not concern ourselves with their emotional state or +moral issues). The result can then serve as a mental scaffolding for building the software implementation. Hierarchical Structure ---------------------- -Like in an economic organization, actors naturally form hierarchies. One actor, -which is to oversee a certain function in the program might want to split up -its task into smaller, more manageable pieces. For this purpose it starts child -actors which it supervises. While the details of supervision are explained -:ref:`here `, we shall concentrate on the underlying concepts in -this section. The only prerequisite is to know that each actor has exactly one +Like in an economic organization, actors naturally form hierarchies. One actor, +which is to oversee a certain function in the program might want to split up +its task into smaller, more manageable pieces. For this purpose it starts child +actors which it supervises. While the details of supervision are explained +:ref:`here `, we shall concentrate on the underlying concepts in +this section. The only prerequisite is to know that each actor has exactly one supervisor, which is the actor that created it. -The quintessential feature of actor systems is that tasks are split up and -delegated until they become small enough to be handled in one piece. In doing -so, not only is the task itself clearly structured, but the resulting actors -can be reasoned about in terms of which messages they should process, how they -should react nominally and how failure should be handled. If one actor does not -have the means for dealing with a certain situation, it sends a corresponding -failure message to its supervisor, asking for help. The recursive structure -then allows to handle failure at the right level. +The quintessential feature of actor systems is that tasks are split up and +delegated until they become small enough to be handled in one piece. In doing +so, not only is the task itself clearly structured, but the resulting actors +can be reasoned about in terms of which messages they should process, how they +should react nominally and how failure should be handled. If one actor does not +have the means for dealing with a certain situation, it sends a corresponding +failure message to its supervisor, asking for help. The recursive structure +then allows to handle failure at the right level. -Compare this to layered software design which easily devolves into defensive -programming with the aim of not leaking any failure out: if the problem is -communicated to the right person, a better solution can be found than if +Compare this to layered software design which easily devolves into defensive +programming with the aim of not leaking any failure out: if the problem is +communicated to the right person, a better solution can be found than if trying to keep everything “under the carpet”. -Now, the difficulty in designing such a system is how to decide who should -supervise what. There is of course no single best solution, but there are a few +Now, the difficulty in designing such a system is how to decide who should +supervise what. There is of course no single best solution, but there are a few guide lines which might be helpful: - - If one actor manages the work another actor is doing, e.g. by passing on - sub-tasks, then the manager should supervise the child. The reason is that - the manager knows which kind of failures are expected and how to handle + - If one actor manages the work another actor is doing, e.g. by passing on + sub-tasks, then the manager should supervise the child. The reason is that + the manager knows which kind of failures are expected and how to handle them. - - If one actor carries very important data (i.e. its state shall not be lost - if avoidable), this actor should source out any possibly dangerous sub-tasks - to children it supervises and handle failures of these children as - appropriate. Depending on the nature of the requests, it may be best to - create a new child for each request, which simplifies state management for - collecting the replies. This is known as the “Error Kernel Pattern” from + - If one actor carries very important data (i.e. its state shall not be lost + if avoidable), this actor should source out any possibly dangerous sub-tasks + to children it supervises and handle failures of these children as + appropriate. Depending on the nature of the requests, it may be best to + create a new child for each request, which simplifies state management for + collecting the replies. This is known as the “Error Kernel Pattern” from Erlang. - - If one actor depends on another actor for carrying out its duty, it should - watch that other actor’s liveness and act upon receiving a termination - notice. This is different from supervision, as the watching party has no - influence on the supervision strategy, and it should be noted that a - functional dependency alone is not a criterion for deciding where to place a + - If one actor depends on another actor for carrying out its duty, it should + watch that other actor’s liveness and act upon receiving a termination + notice. This is different from supervision, as the watching party has no + influence on the supervisor strategy, and it should be noted that a + functional dependency alone is not a criterion for deciding where to place a certain child actor in the hierarchy. -There are of course always exceptions to these rules, but no matter whether you +There are of course always exceptions to these rules, but no matter whether you follow the rules or break them, you should always have a reason. Configuration Container ----------------------- -The actor system as a collaborating ensemble of actors is the natural unit for -managing shared facilities like scheduling services, configuration, logging, -etc. Several actor systems with different configuration may co-exist within the -same JVM without problems, there is no global shared state within Akka itself. -Couple this with the transparent communication between actor systems—within one -node or across a network connection—to see that actor systems themselves can be +The actor system as a collaborating ensemble of actors is the natural unit for +managing shared facilities like scheduling services, configuration, logging, +etc. Several actor systems with different configuration may co-exist within the +same JVM without problems, there is no global shared state within Akka itself. +Couple this with the transparent communication between actor systems—within one +node or across a network connection—to see that actor systems themselves can be used as building blocks in a functional hierarchy. Actor Best Practices -------------------- -#. Actors should be like nice co-workers: do their job efficiently without - bothering everyone else needlessly and avoid hogging resources. Translated - to programming this means to process events and generate responses (or more - requests) in an event-driven manner. Actors should not block (i.e. passively - wait while occupying a Thread) on some external entity, which might be a - lock, a network socket, etc. The blocking operations should be done in some - special-cased thread which sends messages to the actors which shall act on +#. Actors should be like nice co-workers: do their job efficiently without + bothering everyone else needlessly and avoid hogging resources. Translated + to programming this means to process events and generate responses (or more + requests) in an event-driven manner. Actors should not block (i.e. passively + wait while occupying a Thread) on some external entity, which might be a + lock, a network socket, etc. The blocking operations should be done in some + special-cased thread which sends messages to the actors which shall act on them. #. Do not pass mutable objects between actors. In order to ensure that, prefer @@ -94,21 +94,21 @@ Actor Best Practices their mutable state to the outside, you are back in normal Java concurrency land with all the drawbacks. -#. Actors are made to be containers for behavior and state, embracing this - means to not routinely send behavior within messages (which may be tempting - using Scala closures). One of the risks is to accidentally share mutable - state between actors, and this violation of the actor model unfortunately - breaks all the properties which make programming in actors such a nice +#. Actors are made to be containers for behavior and state, embracing this + means to not routinely send behavior within messages (which may be tempting + using Scala closures). One of the risks is to accidentally share mutable + state between actors, and this violation of the actor model unfortunately + breaks all the properties which make programming in actors such a nice experience. What you should not concern yourself with ----------------------------------------- -An actor system manages the resources it is configured to use in order to run -the actors which it contains. There may be millions of actors within one such -system, after all the mantra is to view them as abundant and they weigh in at -an overhead of only roughly 300 bytes per instance. Naturally, the exact order -in which messages are processed in large systems is not controllable by the -application author, but this is also not intended. Take a step back and relax +An actor system manages the resources it is configured to use in order to run +the actors which it contains. There may be millions of actors within one such +system, after all the mantra is to view them as abundant and they weigh in at +an overhead of only roughly 300 bytes per instance. Naturally, the exact order +in which messages are processed in large systems is not controllable by the +application author, but this is also not intended. Take a step back and relax while Akka does the heavy lifting under the hood. diff --git a/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java b/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java index e402d44520..87610a2aef 100644 --- a/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java +++ b/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java @@ -36,6 +36,30 @@ public class FaultHandlingTestBase { //#testkit //#supervisor static public class Supervisor extends UntypedActor { + + //#strategy + private static FaultHandlingStrategy strategy = new OneForOneStrategy(new Function() { + @Override + public Action apply(Throwable t) { + if (t instanceof ArithmeticException) { + return resume(); + } else if (t instanceof NullPointerException) { + return restart(); + } else if (t instanceof IllegalArgumentException) { + return stop(); + } else { + return escalate(); + } + } + }, 10, 60000); + + @Override + public FaultHandlingStrategy supervisorStrategy() { + return strategy; + } + + //#strategy + public void onReceive(Object o) { if (o instanceof Props) { getSender().tell(getContext().actorOf((Props) o)); @@ -44,10 +68,35 @@ public class FaultHandlingTestBase { } } } + //#supervisor - + //#supervisor2 static public class Supervisor2 extends UntypedActor { + + //#strategy2 + private static FaultHandlingStrategy strategy = new OneForOneStrategy(new Function() { + @Override + public Action apply(Throwable t) { + if (t instanceof ArithmeticException) { + return resume(); + } else if (t instanceof NullPointerException) { + return restart(); + } else if (t instanceof IllegalArgumentException) { + return stop(); + } else { + return escalate(); + } + } + }, 10, 60000); + + @Override + public FaultHandlingStrategy supervisorStrategy() { + return strategy; + } + + //#strategy2 + public void onReceive(Object o) { if (o instanceof Props) { getSender().tell(getContext().actorOf((Props) o)); @@ -55,18 +104,19 @@ public class FaultHandlingTestBase { unhandled(o); } } - + @Override public void preRestart(Throwable cause, Option msg) { // do not kill all children, which is the default here } } + //#supervisor2 - + //#child static public class Child extends UntypedActor { int state = 0; - + public void onReceive(Object o) throws Exception { if (o instanceof Exception) { throw (Exception) o; @@ -79,39 +129,23 @@ public class FaultHandlingTestBase { } } } + //#child - - //#strategy - static FaultHandlingStrategy strategy = new OneForOneStrategy(new Function() { - @Override - public Action apply(Throwable t) { - if (t instanceof ArithmeticException) { - return resume(); - } else if (t instanceof NullPointerException) { - return restart(); - } else if (t instanceof IllegalArgumentException) { - return stop(); - } else { - return escalate(); - } - } - }, 10, 60000); - //#strategy - + //#testkit static ActorSystem system; Duration timeout = Duration.create(5, SECONDS); - + @BeforeClass public static void start() { system = ActorSystem.create("test", AkkaSpec.testConf()); } - + @AfterClass public static void cleanup() { system.shutdown(); } - + @Test public void mustEmployFaultHandler() { // code here @@ -122,32 +156,32 @@ public class FaultHandlingTestBase { EventFilter ex4 = (EventFilter) new ErrorFilter(Exception.class); Seq ignoreExceptions = seq(ex1, ex2, ex3, ex4); system.eventStream().publish(new TestEvent.Mute(ignoreExceptions)); - + //#create - Props superprops = new Props(Supervisor.class).withFaultHandler(strategy); + Props superprops = new Props(Supervisor.class); ActorRef supervisor = system.actorOf(superprops, "supervisor"); ActorRef child = (ActorRef) Await.result(supervisor.ask(new Props(Child.class), 5000), timeout); //#create - + //#resume child.tell(42); assert Await.result(child.ask("get", 5000), timeout).equals(42); child.tell(new ArithmeticException()); assert Await.result(child.ask("get", 5000), timeout).equals(42); //#resume - + //#restart child.tell(new NullPointerException()); assert Await.result(child.ask("get", 5000), timeout).equals(0); //#restart - + //#stop final TestProbe probe = new TestProbe(system); probe.watch(child); child.tell(new IllegalArgumentException()); probe.expectMsg(new Terminated(child)); //#stop - + //#escalate-kill child = (ActorRef) Await.result(supervisor.ask(new Props(Child.class), 5000), timeout); probe.watch(child); @@ -155,9 +189,9 @@ public class FaultHandlingTestBase { child.tell(new Exception()); probe.expectMsg(new Terminated(child)); //#escalate-kill - + //#escalate-restart - superprops = new Props(Supervisor2.class).withFaultHandler(strategy); + superprops = new Props(Supervisor2.class); supervisor = system.actorOf(superprops, "supervisor2"); child = (ActorRef) Await.result(supervisor.ask(new Props(Child.class), 5000), timeout); child.tell(23); @@ -167,10 +201,11 @@ public class FaultHandlingTestBase { //#escalate-restart //#testkit } -//#testkit + + //#testkit public Seq seq(A... args) { return JavaConverters.collectionAsScalaIterableConverter(java.util.Arrays.asList(args)).asScala().toSeq(); } -//#testkit + //#testkit } //#testkit \ No newline at end of file diff --git a/akka-docs/java/fault-tolerance.rst b/akka-docs/java/fault-tolerance.rst index ca6481bb77..4d0b1abe8c 100644 --- a/akka-docs/java/fault-tolerance.rst +++ b/akka-docs/java/fault-tolerance.rst @@ -8,9 +8,9 @@ Fault Handling Strategies (Java) .. contents:: :local: As explained in :ref:`actor-systems` each actor is the supervisor of its -children, and as such each actor is given a fault handling strategy when it is -created. This strategy cannot be changed afterwards as it is an integral part -of the actor system’s structure. +children, and as such each actor defines fault handling supervisor strategy. +This strategy cannot be changed afterwards as it is an integral part of the +actor system’s structure. Creating a Fault Handling Strategy ---------------------------------- @@ -26,7 +26,7 @@ First off, it is a one-for-one strategy, meaning that each child is treated separately (an all-for-one strategy works very similarly, the only difference is that any decision is applied to all children of the supervisor, not only the failing one). There are limits set on the restart frequency, namely maximum 10 -restarts per minute; each of these settings defaults to could be left out, which means +restarts per minute; each of these settings could be left out, which means that the respective limit does not apply, leaving the possibility to specify an absolute upper limit on the restarts or to make the restarts work infinitely. @@ -50,7 +50,7 @@ where ``TestProbe`` provides an actor ref useful for receiving and inspecting re .. includecode:: code/akka/docs/actor/FaultHandlingTestBase.java :include: testkit -Using the strategy shown above let us create actors: +Let us create actors: .. includecode:: code/akka/docs/actor/FaultHandlingTestBase.java :include: create diff --git a/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala b/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala index 6fb4d3c36b..dc9159de43 100644 --- a/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala @@ -17,6 +17,18 @@ object FaultHandlingDocSpec { //#supervisor //#supervisor class Supervisor extends Actor { + //#strategy + import akka.actor.OneForOneStrategy + import akka.actor.FaultHandlingStrategy._ + + override val supervisorStrategy = OneForOneStrategy({ + case _: ArithmeticException ⇒ Resume + case _: NullPointerException ⇒ Restart + case _: IllegalArgumentException ⇒ Stop + case _: Exception ⇒ Escalate + }: Decider, maxNrOfRetries = Some(10), withinTimeRange = Some(60000)) + //#strategy + def receive = { case p: Props ⇒ sender ! context.actorOf(p) } @@ -25,6 +37,18 @@ object FaultHandlingDocSpec { //#supervisor2 class Supervisor2 extends Actor { + //#strategy2 + import akka.actor.OneForOneStrategy + import akka.actor.FaultHandlingStrategy._ + + override val supervisorStrategy = OneForOneStrategy({ + case _: ArithmeticException ⇒ Resume + case _: NullPointerException ⇒ Restart + case _: IllegalArgumentException ⇒ Stop + case _: Exception ⇒ Escalate + }: Decider, maxNrOfRetries = Some(10), withinTimeRange = Some(60000)) + //#strategy2 + def receive = { case p: Props ⇒ sender ! context.actorOf(p) } @@ -56,21 +80,9 @@ class FaultHandlingDocSpec extends AkkaSpec with ImplicitSender { "apply the chosen strategy for its child" in { //#testkit - //#strategy - import akka.actor.OneForOneStrategy - import akka.actor.FaultHandlingStrategy._ - val strategy = OneForOneStrategy({ - case _: ArithmeticException ⇒ Resume - case _: NullPointerException ⇒ Restart - case _: IllegalArgumentException ⇒ Stop - case _: Exception ⇒ Escalate - }: Decider, maxNrOfRetries = Some(10), withinTimeRange = Some(60000)) - - //#strategy //#create - val superprops = Props[Supervisor].withFaultHandler(strategy) - val supervisor = system.actorOf(superprops, "supervisor") + val supervisor = system.actorOf(Props[Supervisor], "supervisor") supervisor ! Props[Child] val child = expectMsgType[ActorRef] // retrieve answer from TestKit’s testActor @@ -114,8 +126,7 @@ class FaultHandlingDocSpec extends AkkaSpec with ImplicitSender { expectMsg(Terminated(child2)) //#escalate-kill //#escalate-restart - val superprops2 = Props[Supervisor2].withFaultHandler(strategy) - val supervisor2 = system.actorOf(superprops2, "supervisor2") + val supervisor2 = system.actorOf(Props[Supervisor2], "supervisor2") supervisor2 ! Props[Child] val child3 = expectMsgType[ActorRef] diff --git a/akka-docs/scala/fault-tolerance.rst b/akka-docs/scala/fault-tolerance.rst index c2ffa10c79..a4cdd584f0 100644 --- a/akka-docs/scala/fault-tolerance.rst +++ b/akka-docs/scala/fault-tolerance.rst @@ -8,9 +8,9 @@ Fault Handling Strategies (Scala) .. contents:: :local: As explained in :ref:`actor-systems` each actor is the supervisor of its -children, and as such each actor is given a fault handling strategy when it is -created. This strategy cannot be changed afterwards as it is an integral part -of the actor system’s structure. +children, and as such each actor defines fault handling supervisor strategy. +This strategy cannot be changed afterwards as it is an integral part of the +actor system’s structure. Creating a Fault Handling Strategy ---------------------------------- @@ -56,7 +56,7 @@ MustMatchers`` .. includecode:: code/akka/docs/actor/FaultHandlingDocSpec.scala :include: testkit -Using the strategy shown above let us create actors: +Let us create actors: .. includecode:: code/akka/docs/actor/FaultHandlingDocSpec.scala :include: create diff --git a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala index ad5b572ec3..97df807119 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala @@ -181,8 +181,10 @@ class TestActorRefSpec extends AkkaSpec("disp1.type=Dispatcher") with BeforeAndA override def postRestart(reason: Throwable) { counter -= 1 } }), self, "child") + override def supervisorStrategy = OneForOneStrategy(List(classOf[ActorKilledException]), 5, 1000) + def receiveT = { case "sendKill" ⇒ ref ! Kill } - }).withFaultHandler(OneForOneStrategy(List(classOf[ActorKilledException]), 5, 1000))) + })) boss ! "sendKill" From 61da9b1890d0c6779b5eaf4c5551512e245e7cbe Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 23 Jan 2012 13:54:11 +0100 Subject: [PATCH 097/152] Improving logging docs --- akka-docs/java/logging.rst | 89 +++++++++++++++++++++++++++++++ akka-docs/scala/logging.rst | 101 ++++++++++++++++++++++++++++++++++++ 2 files changed, 190 insertions(+) diff --git a/akka-docs/java/logging.rst b/akka-docs/java/logging.rst index ffee92d00e..43cad5c0d3 100644 --- a/akka-docs/java/logging.rst +++ b/akka-docs/java/logging.rst @@ -40,6 +40,95 @@ treatment of this case, e.g. in the SLF4J event listener which will then use the string instead of the class’ name for looking up the logger instance to use. +Auxiliary logging options +------------------------- + +Akka has a couple of configuration options for very low level debugging, that makes most sense in +for developers and not for operations. + +This config option is very good if you want to know what config settings are loaded by Akka: + +.. code-block:: ruby + + akka { + # Log the complete configuration at INFO level when the actor system is started. + # This is useful when you are uncertain of what configuration is used. + logConfigOnStart = off + } + +If you want very detailed logging of all automatically received messages that are processed +by Actors: + +.. code-block:: ruby + + akka { + debug { + # enable DEBUG logging of all AutoReceiveMessages (Kill, PoisonPill and the like) + autoreceive = off + } + } + +If you want very detailed logging of all lifecycle changes of Actors (restarts, deaths etc): + +.. code-block:: ruby + + akka { + debug { + # enable DEBUG logging of actor lifecycle changes + lifecycle = off + } + } + +If you want very detailed logging of all events, transitions and timers of FSM Actors that extend LoggingFSM: + +.. code-block:: ruby + + akka { + debug { + # enable DEBUG logging of all LoggingFSMs for events, transitions and timers + fsm = off + } + } + +If you want to monitor subscriptions (subscribe/unsubscribe) on the ActorSystem.eventStream: + +.. code-block:: ruby + + akka { + debug { + # enable DEBUG logging of subscription changes on the eventStream + event-stream = off + } + } + +Auxiliary remote logging options +-------------------------------- + +If you want to see all messages that are sent through remoting at DEBUG log level: +(This is logged as they are sent by the transport layer, not by the Actor) + +.. code-block:: ruby + + akka { + remote { + # If this is "on", Akka will log all outbound messages at DEBUG level, if off then they are not logged + log-sent-messages = off + } + } + +If you want to see all messages that are received through remoting at DEBUG log level: +(This is logged as they are received by the transport layer, not by any Actor) + +.. code-block:: ruby + + akka { + remote { + # If this is "on", Akka will log all inbound messages at DEBUG level, if off then they are not logged + log-received-messages = off + } + } + + Event Handler ============= diff --git a/akka-docs/scala/logging.rst b/akka-docs/scala/logging.rst index debafcedc5..7ed8f4e7d9 100644 --- a/akka-docs/scala/logging.rst +++ b/akka-docs/scala/logging.rst @@ -37,6 +37,107 @@ The source object is translated to a String according to the following rules: The log message may contain argument placeholders ``{}``, which will be substituted if the log level is enabled. +Auxiliary logging options +------------------------- + +Akka has a couple of configuration options for very low level debugging, that makes most sense in +for developers and not for operations. + +This config option is very good if you want to know what config settings are loaded by Akka: + +.. code-block:: ruby + + akka { + # Log the complete configuration at INFO level when the actor system is started. + # This is useful when you are uncertain of what configuration is used. + logConfigOnStart = off + } + +If you want very detailed logging of all user-level messages that are processed +by Actors that use akka.event.LoggingReceive: + +.. code-block:: ruby + + akka { + debug { + # enable function of LoggingReceive, which is to log any received message at + # DEBUG level + receive = off + } + } + +If you want very detailed logging of all automatically received messages that are processed +by Actors: + +.. code-block:: ruby + + akka { + debug { + # enable DEBUG logging of all AutoReceiveMessages (Kill, PoisonPill and the like) + autoreceive = off + } + } + +If you want very detailed logging of all lifecycle changes of Actors (restarts, deaths etc): + +.. code-block:: ruby + + akka { + debug { + # enable DEBUG logging of actor lifecycle changes + lifecycle = off + } + } + +If you want very detailed logging of all events, transitions and timers of FSM Actors that extend LoggingFSM: + +.. code-block:: ruby + + akka { + debug { + # enable DEBUG logging of all LoggingFSMs for events, transitions and timers + fsm = off + } + } + +If you want to monitor subscriptions (subscribe/unsubscribe) on the ActorSystem.eventStream: + +.. code-block:: ruby + + akka { + debug { + # enable DEBUG logging of subscription changes on the eventStream + event-stream = off + } + } + +Auxiliary remote logging options +-------------------------------- + +If you want to see all messages that are sent through remoting at DEBUG log level: +(This is logged as they are sent by the transport layer, not by the Actor) + +.. code-block:: ruby + + akka { + remote { + # If this is "on", Akka will log all outbound messages at DEBUG level, if off then they are not logged + log-sent-messages = off + } + } + +If you want to see all messages that are received through remoting at DEBUG log level: +(This is logged as they are received by the transport layer, not by any Actor) + +.. code-block:: ruby + + akka { + remote { + # If this is "on", Akka will log all inbound messages at DEBUG level, if off then they are not logged + log-received-messages = off + } + } + Translating Log Source to String and Class ------------------------------------------ From 49488c267058f2ef10f0da39dbcf96ec7b3da8fe Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 23 Jan 2012 14:14:39 +0100 Subject: [PATCH 098/152] Changing logging docs to have settings turned on --- akka-docs/java/logging.rst | 14 +++++++------- akka-docs/scala/logging.rst | 16 ++++++++-------- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/akka-docs/java/logging.rst b/akka-docs/java/logging.rst index 43cad5c0d3..65e840b271 100644 --- a/akka-docs/java/logging.rst +++ b/akka-docs/java/logging.rst @@ -53,7 +53,7 @@ This config option is very good if you want to know what config settings are loa akka { # Log the complete configuration at INFO level when the actor system is started. # This is useful when you are uncertain of what configuration is used. - logConfigOnStart = off + logConfigOnStart = on } If you want very detailed logging of all automatically received messages that are processed @@ -64,7 +64,7 @@ by Actors: akka { debug { # enable DEBUG logging of all AutoReceiveMessages (Kill, PoisonPill and the like) - autoreceive = off + autoreceive = on } } @@ -75,7 +75,7 @@ If you want very detailed logging of all lifecycle changes of Actors (restarts, akka { debug { # enable DEBUG logging of actor lifecycle changes - lifecycle = off + lifecycle = on } } @@ -86,7 +86,7 @@ If you want very detailed logging of all events, transitions and timers of FSM A akka { debug { # enable DEBUG logging of all LoggingFSMs for events, transitions and timers - fsm = off + fsm = on } } @@ -97,7 +97,7 @@ If you want to monitor subscriptions (subscribe/unsubscribe) on the ActorSystem. akka { debug { # enable DEBUG logging of subscription changes on the eventStream - event-stream = off + event-stream = on } } @@ -112,7 +112,7 @@ If you want to see all messages that are sent through remoting at DEBUG log leve akka { remote { # If this is "on", Akka will log all outbound messages at DEBUG level, if off then they are not logged - log-sent-messages = off + log-sent-messages = on } } @@ -124,7 +124,7 @@ If you want to see all messages that are received through remoting at DEBUG log akka { remote { # If this is "on", Akka will log all inbound messages at DEBUG level, if off then they are not logged - log-received-messages = off + log-received-messages = on } } diff --git a/akka-docs/scala/logging.rst b/akka-docs/scala/logging.rst index 7ed8f4e7d9..0cc969bad7 100644 --- a/akka-docs/scala/logging.rst +++ b/akka-docs/scala/logging.rst @@ -50,7 +50,7 @@ This config option is very good if you want to know what config settings are loa akka { # Log the complete configuration at INFO level when the actor system is started. # This is useful when you are uncertain of what configuration is used. - logConfigOnStart = off + logConfigOnStart = on } If you want very detailed logging of all user-level messages that are processed @@ -62,7 +62,7 @@ by Actors that use akka.event.LoggingReceive: debug { # enable function of LoggingReceive, which is to log any received message at # DEBUG level - receive = off + receive = on } } @@ -74,7 +74,7 @@ by Actors: akka { debug { # enable DEBUG logging of all AutoReceiveMessages (Kill, PoisonPill and the like) - autoreceive = off + autoreceive = on } } @@ -85,7 +85,7 @@ If you want very detailed logging of all lifecycle changes of Actors (restarts, akka { debug { # enable DEBUG logging of actor lifecycle changes - lifecycle = off + lifecycle = on } } @@ -96,7 +96,7 @@ If you want very detailed logging of all events, transitions and timers of FSM A akka { debug { # enable DEBUG logging of all LoggingFSMs for events, transitions and timers - fsm = off + fsm = on } } @@ -107,7 +107,7 @@ If you want to monitor subscriptions (subscribe/unsubscribe) on the ActorSystem. akka { debug { # enable DEBUG logging of subscription changes on the eventStream - event-stream = off + event-stream = on } } @@ -122,7 +122,7 @@ If you want to see all messages that are sent through remoting at DEBUG log leve akka { remote { # If this is "on", Akka will log all outbound messages at DEBUG level, if off then they are not logged - log-sent-messages = off + log-sent-messages = on } } @@ -134,7 +134,7 @@ If you want to see all messages that are received through remoting at DEBUG log akka { remote { # If this is "on", Akka will log all inbound messages at DEBUG level, if off then they are not logged - log-received-messages = off + log-received-messages = on } } From 690e87441f7b41daf56b87f25338b73b3c8f1c1c Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 23 Jan 2012 14:35:01 +0100 Subject: [PATCH 099/152] Adding DEBUG documentation and reference to testkit --- akka-docs/java/logging.rst | 10 ++++++++++ akka-docs/scala/logging.rst | 10 ++++++++++ 2 files changed, 20 insertions(+) diff --git a/akka-docs/java/logging.rst b/akka-docs/java/logging.rst index 65e840b271..7166108306 100644 --- a/akka-docs/java/logging.rst +++ b/akka-docs/java/logging.rst @@ -46,6 +46,14 @@ Auxiliary logging options Akka has a couple of configuration options for very low level debugging, that makes most sense in for developers and not for operations. +You almost definitely need to have logging set to DEBUG to use any of the options below: + +.. code-block:: ruby + + akka { + loglevel = DEBUG + } + This config option is very good if you want to know what config settings are loaded by Akka: .. code-block:: ruby @@ -128,6 +136,8 @@ If you want to see all messages that are received through remoting at DEBUG log } } +Also see the logging options for TestKit: :ref:`actor.logging`. + Event Handler ============= diff --git a/akka-docs/scala/logging.rst b/akka-docs/scala/logging.rst index 0cc969bad7..908017e8b8 100644 --- a/akka-docs/scala/logging.rst +++ b/akka-docs/scala/logging.rst @@ -43,6 +43,14 @@ Auxiliary logging options Akka has a couple of configuration options for very low level debugging, that makes most sense in for developers and not for operations. +You almost definitely need to have logging set to DEBUG to use any of the options below: + +.. code-block:: ruby + + akka { + loglevel = DEBUG + } + This config option is very good if you want to know what config settings are loaded by Akka: .. code-block:: ruby @@ -138,6 +146,8 @@ If you want to see all messages that are received through remoting at DEBUG log } } +Also see the logging options for TestKit: :ref:`actor.logging`. + Translating Log Source to String and Class ------------------------------------------ From 5e11b2df9dca2aa34b3f3c696d2012ba3407f50e Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 23 Jan 2012 15:37:43 +0100 Subject: [PATCH 100/152] add FSMDocSpec and improve FSM API MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - don’t require Option[Duration] and its implicit conversion - provider “->” extractor inside the FSM trait --- .../src/main/scala/akka/actor/FSM.scala | 30 +++--- .../code/akka/docs/actor/FSMDocSpec.scala | 93 +++++++++++++++++++ .../main/scala/akka/testkit/TestFSMRef.scala | 4 +- 3 files changed, 109 insertions(+), 18 deletions(-) create mode 100644 akka-docs/scala/code/akka/docs/actor/FSMDocSpec.scala diff --git a/akka-actor/src/main/scala/akka/actor/FSM.scala b/akka-actor/src/main/scala/akka/actor/FSM.scala index bf20b43274..ee9d38efab 100644 --- a/akka-actor/src/main/scala/akka/actor/FSM.scala +++ b/akka-actor/src/main/scala/akka/actor/FSM.scala @@ -49,21 +49,6 @@ object FSM { } } - /* - * This extractor is just convenience for matching a (S, S) pair, including a - * reminder what the new state is. - */ - object -> { - def unapply[S](in: (S, S)) = Some(in) - } - - /* - * With these implicits in scope, you can write "5 seconds" anywhere a - * Duration or Option[Duration] is expected. This is conveniently true - * for derived classes. - */ - implicit def d2od(d: Duration): Option[Duration] = Some(d) - case class LogEntry[S, D](stateName: S, stateData: D, event: Any) case class State[S, D](stateName: S, stateData: D, timeout: Option[Duration] = None, stopReason: Option[Reason] = None, replies: List[Any] = Nil) { @@ -208,7 +193,12 @@ trait FSM[S, D] extends Listeners { * @param stateTimeout default state timeout for this state * @param stateFunction partial function describing response to input */ - protected final def when(stateName: S, stateTimeout: Timeout = None)(stateFunction: StateFunction) = { + protected final def when(stateName: S, stateTimeout: Duration = null)(stateFunction: StateFunction) = { + 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) = { register(stateName, stateFunction, stateTimeout) } @@ -312,6 +302,14 @@ trait FSM[S, D] extends Listeners { stateTimeouts(state) = timeout } + /** + * This extractor is just convenience for matching a (S, S) pair, including a + * reminder what the new state is. + */ + object -> { + def unapply[S](in: (S, S)) = Some(in) + } + /** * Set handler which is called upon each state transition, i.e. not when * staying in the same state. This may use the pair extractor defined in the diff --git a/akka-docs/scala/code/akka/docs/actor/FSMDocSpec.scala b/akka-docs/scala/code/akka/docs/actor/FSMDocSpec.scala new file mode 100644 index 0000000000..5668d88868 --- /dev/null +++ b/akka-docs/scala/code/akka/docs/actor/FSMDocSpec.scala @@ -0,0 +1,93 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.docs.actor + +import akka.testkit.AkkaSpec + +class FSMDocSpec extends AkkaSpec { + + "simple finite state machine" must { + //#simple-imports + import akka.actor.{ Actor, ActorRef, FSM, Props } + import akka.util.duration._ + //#simple-imports + //#simple-events + // received events + case class SetTarget(ref: ActorRef) + case class Queue(obj: Any) + case object Flush + + // sent events + case class Batch(obj: Seq[Any]) + //#simple-events + //#simple-state + // states + trait State + case object Idle extends State + case object Active extends State + + trait Data + case object Uninitialized extends Data + case class Todo(target: ActorRef, queue: Seq[Any]) extends Data + //#simple-state + //#simple-fsm + class Buncher extends Actor with FSM[State, Data] { + + startWith(Idle, Uninitialized) + + when(Idle) { + case Event(SetTarget(ref), Uninitialized) ⇒ stay using Todo(ref, Vector.empty) + } + + //#simple-transition + onTransition { + case Active -> Idle ⇒ + stateData match { + case Todo(ref, queue) ⇒ ref ! Batch(queue) + } + } + //#simple-transition + + when(Active, stateTimeout = 1 second) { + case Event(Flush | FSM.StateTimeout, t: Todo) ⇒ goto(Idle) using t.copy(queue = Vector.empty) + } + + //#simple-unhandled + whenUnhandled { + // common code for both states + case Event(Queue(obj), t @ Todo(_, v)) ⇒ + goto(Active) using t.copy(queue = v :+ obj) + + case Event(e, s) ⇒ + log.warning("received unhandled request {} in state {}/{}", e, stateName, s) + stay + } + //#simple-unhandled + + initialize + } + //#simple-fsm + + "batch correctly" in { + val buncher = system.actorOf(Props(new Buncher)) + buncher ! SetTarget(testActor) + buncher ! Queue(42) + buncher ! Queue(43) + expectMsg(Batch(Seq(42, 43))) + buncher ! Queue(44) + buncher ! Flush + buncher ! Queue(45) + expectMsg(Batch(Seq(44))) + expectMsg(Batch(Seq(45))) + } + + "batch not if uninitialized" in { + val buncher = system.actorOf(Props(new Buncher)) + buncher ! Queue(42) + expectNoMsg + } + + } + +} \ No newline at end of file diff --git a/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala index f486e3a5bb..fd3567f19c 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala @@ -62,8 +62,8 @@ class TestFSMRef[S, D, T <: Actor]( * corresponding transition initiated from within the FSM, including timeout * and stop handling. */ - def setState(stateName: S = fsm.stateName, stateData: D = fsm.stateData, timeout: Option[Duration] = None, stopReason: Option[FSM.Reason] = None) { - fsm.applyState(FSM.State(stateName, stateData, timeout, stopReason)) + def setState(stateName: S = fsm.stateName, stateData: D = fsm.stateData, timeout: Duration = null, stopReason: Option[FSM.Reason] = None) { + fsm.applyState(FSM.State(stateName, stateData, Option(timeout), stopReason)) } /** From 3a30f915c3bfb626c8d506a85e96b3957db1944a Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 23 Jan 2012 15:59:18 +0100 Subject: [PATCH 101/152] remove use of `akka.actor.timeout` property for `ask` --- .../akka/migration/AskableActorRef.scala | 2 +- .../src/test/scala/akka/pattern/AskSpec.scala | 3 +- .../main/scala/akka/pattern/AskSupport.scala | 4 +-- .../main/scala/akka/pattern/Patterns.scala | 33 ------------------- .../src/main/scala/akka/pattern/package.scala | 17 +++++----- .../docs/actor/UntypedActorDocTestBase.java | 4 +-- .../project/migration-guide-1.3.x-2.0.x.rst | 15 ++++++--- akka-docs/scala/actors.rst | 3 -- 8 files changed, 25 insertions(+), 56 deletions(-) diff --git a/akka-actor-migration/src/main/scala/akka/migration/AskableActorRef.scala b/akka-actor-migration/src/main/scala/akka/migration/AskableActorRef.scala index caf8921812..fc4f28cd8b 100644 --- a/akka-actor-migration/src/main/scala/akka/migration/AskableActorRef.scala +++ b/akka-actor-migration/src/main/scala/akka/migration/AskableActorRef.scala @@ -37,7 +37,7 @@ class AskableActorRef(val actorRef: ActorRef) { * * [see the [[akka.dispatch.Future]] companion object for a description of `flow`] */ - def ask(message: Any)(implicit timeout: Timeout = null): Future[Any] = akka.pattern.ask(actorRef, message)(timeout) + def ask(message: Any)(implicit timeout: Timeout): Future[Any] = akka.pattern.ask(actorRef, message)(timeout) /** * Sends a message asynchronously and returns a [[akka.dispatch.Future]] diff --git a/akka-actor-tests/src/test/scala/akka/pattern/AskSpec.scala b/akka-actor-tests/src/test/scala/akka/pattern/AskSpec.scala index 44cdd91eba..ecb9690594 100644 --- a/akka-actor-tests/src/test/scala/akka/pattern/AskSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/pattern/AskSpec.scala @@ -4,6 +4,7 @@ package akka.pattern import akka.testkit.AkkaSpec +import akka.util.duration._ class AskSpec extends AkkaSpec { @@ -11,7 +12,7 @@ class AskSpec extends AkkaSpec { "return broken promises on DeadLetters" in { val dead = system.actorFor("/system/deadLetters") - val f = dead ask 42 + val f = dead.ask(42)(1 second) f.isCompleted must be(true) f.value.get match { case Left(_: AskTimeoutException) ⇒ diff --git a/akka-actor/src/main/scala/akka/pattern/AskSupport.scala b/akka-actor/src/main/scala/akka/pattern/AskSupport.scala index 57dec1debe..492bb46ed8 100644 --- a/akka-actor/src/main/scala/akka/pattern/AskSupport.scala +++ b/akka-actor/src/main/scala/akka/pattern/AskSupport.scala @@ -51,7 +51,7 @@ object AskSupport { * * [see the [[akka.dispatch.Future]] companion object for a description of `flow`] */ - def ask(message: Any)(implicit timeout: Timeout = null): Future[Any] = akka.pattern.ask(actorRef, message)(timeout) + def ask(message: Any)(implicit timeout: Timeout): Future[Any] = akka.pattern.ask(actorRef, message)(timeout) /** * Sends a message asynchronously and returns a [[akka.dispatch.Future]] @@ -81,7 +81,7 @@ object AskSupport { * * [see the [[akka.dispatch.Future]] companion object for a description of `flow`] */ - def ?(message: Any)(implicit timeout: Timeout = null): Future[Any] = akka.pattern.ask(actorRef, message)(timeout) + def ?(message: Any)(implicit timeout: Timeout): Future[Any] = akka.pattern.ask(actorRef, message)(timeout) } /** diff --git a/akka-actor/src/main/scala/akka/pattern/Patterns.scala b/akka-actor/src/main/scala/akka/pattern/Patterns.scala index b1498ee2d0..8c52f096f1 100644 --- a/akka-actor/src/main/scala/akka/pattern/Patterns.scala +++ b/akka-actor/src/main/scala/akka/pattern/Patterns.scala @@ -9,39 +9,6 @@ object Patterns { import akka.pattern.{ ask ⇒ scalaAsk } import akka.util.{ Timeout, Duration } - /** - * Java API for `akka.pattern.ask`: - * 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 - * given timeout has expired; this is independent from any timeout applied - * while awaiting a result for this future (i.e. in - * `Await.result(..., timeout)`). - * - * This variant will use the `akka.actor.timeout` from the configuration. - * - * Warning: - * When using future callbacks, inside actors you need to carefully avoid closing over - * the containing actor’s object, i.e. do not call methods or access mutable state - * on the enclosing actor from within the callback. This would break the actor - * encapsulation and may introduce synchronization bugs and race conditions because - * the callback will be scheduled concurrently to the enclosing actor. Unfortunately - * there is not yet a way to detect these illegal accesses at compile time. - * - * Recommended usage: - * - * {{{ - * final Future f = Patterns.ask(worker, request, timeout); - * f.onSuccess(new Procedure() { - * public void apply(Object o) { - * nextActor.tell(new EnrichedResult(request, o)); - * } - * }); - * }}} - */ - def ask(actor: ActorRef, message: Any): Future[AnyRef] = scalaAsk(actor, message).asInstanceOf[Future[AnyRef]] - /** * Java API for `akka.pattern.ask`: * Sends a message asynchronously and returns a [[akka.dispatch.Future]] diff --git a/akka-actor/src/main/scala/akka/pattern/package.scala b/akka-actor/src/main/scala/akka/pattern/package.scala index 4abb7c41d7..2689b72548 100644 --- a/akka-actor/src/main/scala/akka/pattern/package.scala +++ b/akka-actor/src/main/scala/akka/pattern/package.scala @@ -87,20 +87,19 @@ package object pattern { * * [see [[akka.dispatch.Future]] for a description of `flow`] */ - def ask(actorRef: ActorRef, message: Any)(implicit timeout: Timeout = null): Future[Any] = actorRef match { + def ask(actorRef: ActorRef, message: Any)(implicit timeout: Timeout): Future[Any] = actorRef match { case ref: InternalActorRef if ref.isTerminated ⇒ actorRef.tell(message) Promise.failed(new AskTimeoutException("sending to terminated ref breaks promises"))(ref.provider.dispatcher) case ref: InternalActorRef ⇒ val provider = ref.provider - (if (timeout == null) provider.settings.ActorTimeout else timeout) match { - case t if t.duration.length <= 0 ⇒ - actorRef.tell(message) - Promise.failed(new AskTimeoutException("not asking with negative timeout"))(provider.dispatcher) - case t ⇒ - val a = AskSupport.createAsker(provider, t) - actorRef.tell(message, a) - a.result + if (timeout.duration.length <= 0) { + actorRef.tell(message) + Promise.failed(new AskTimeoutException("not asking with negative timeout"))(provider.dispatcher) + } else { + val a = AskSupport.createAsker(provider, timeout) + actorRef.tell(message, a) + a.result } case _ ⇒ throw new IllegalArgumentException("incompatible ActorRef " + actorRef) } diff --git a/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java b/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java index eb8245ee57..a72c828862 100644 --- a/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java +++ b/akka-docs/java/code/akka/docs/actor/UntypedActorDocTestBase.java @@ -231,8 +231,8 @@ public class UntypedActorDocTestBase { final Timeout t = new Timeout(Duration.create(5, TimeUnit.SECONDS)); final ArrayList> futures = new ArrayList>(); - futures.add(ask(actorA, "request")); // using `akka.actor.timeout` from config - futures.add(ask(actorB, "reqeest", t)); // using explicit timeout from above + futures.add(ask(actorA, "request", 1000)); // using 1000ms timeout + futures.add(ask(actorB, "reqeest", t)); // using timeout from above final Future> aggregate = Futures.sequence(futures, system.dispatcher()); diff --git a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst index 273d44c1d1..4dd9b8cc39 100644 --- a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst +++ b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst @@ -207,6 +207,9 @@ reply to be received; it is independent of the timeout applied when awaiting completion of the :class:`Future`, however, the actor will complete the :class:`Future` with an :class:`AskTimeoutException` when it stops itself. +Since there is no good library default value for the ask-timeout, specification +of a timeout is required for all usages as shown below. + Also, since the ``ask`` feature is coupling futures and actors, it is no longer offered on the :class:`ActorRef` itself, but instead as a use pattern to be imported. While Scala’s implicit conversions enable transparent replacement, @@ -221,15 +224,17 @@ v2.0 (Scala):: import akka.pattern.ask - actorRef ? message - ask(actorRef, message) // will use `akka.actor.timeout` or implicit Timeout - ask(actorRef, message)(timeout) + implicit val timeout: Timeout = ... + actorRef ? message // uses implicit timeout + actorRef ask message // uses implicit timeout + actorRef.ask(message)(timeout) // uses explicit timeout + ask(actorRef, message) // uses implicit timeout + ask(actorRef, message)(timeout) // uses explicit timeout v2.0 (Java):: import akka.pattern.Patterns; - Patterns.ask(actorRef, message) // will use `akka.actor.timeout` Patterns.ask(actorRef, message, timeout) Documentation: @@ -358,7 +363,7 @@ v2.0:: import akka.event.Logging val log = Logging(context.system, this) // will include system name in message source - val log = Logging(system.eventStream, this) // will not include system name + val log = Logging(system.eventStream, getClass.getName) // will not include system name log.error(exception, message) log.warning(message) log.info(message) diff --git a/akka-docs/scala/actors.rst b/akka-docs/scala/actors.rst index da0ae5b91e..fe1b134b65 100644 --- a/akka-docs/scala/actors.rst +++ b/akka-docs/scala/actors.rst @@ -389,9 +389,6 @@ taken from one of the following locations in order of precedence: .. includecode:: code/akka/docs/actor/ActorDocSpec.scala#using-implicit-timeout -3. actor system’s default value from ``akka.actor.timeout`` setting for - :meth:`ask` methods - See :ref:`futures-scala` for more information on how to await or query a future. From 46409b2f48c0bc9c45e28a1a528d77974c9afa0b Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 23 Jan 2012 15:59:08 +0100 Subject: [PATCH 102/152] Adding 0MQ to the microkernel --- project/AkkaBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 1d7413d201..9e5e773911 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -31,7 +31,7 @@ object AkkaBuild extends Build { Unidoc.unidocExclude := Seq(samples.id, tutorials.id), Dist.distExclude := Seq(actorTests.id, akkaSbtPlugin.id, docs.id) ), - aggregate = Seq(actor, testkit, actorTests, remote, slf4j, agent, transactor, mailboxes, kernel, akkaSbtPlugin, actorMigration, samples, tutorials, docs) + aggregate = Seq(actor, testkit, actorTests, remote, slf4j, agent, transactor, mailboxes, zeroMQ, kernel, akkaSbtPlugin, actorMigration, samples, tutorials, docs) ) lazy val actor = Project( From 1572da09db257e3ac058c72ad9fe7d29d91a4af1 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 23 Jan 2012 16:11:58 +0100 Subject: [PATCH 103/152] Changed name of the instance field as suggested. See #1702 --- .../src/test/java/akka/actor/JavaExtension.java | 8 ++++---- .../code/akka/docs/extension/ExtensionDocTestBase.java | 10 ++++++---- .../docs/extension/SettingsExtensionDocTestBase.java | 10 ++++++---- 3 files changed, 16 insertions(+), 12 deletions(-) diff --git a/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java b/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java index 734e6fa506..812e79e287 100644 --- a/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java +++ b/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java @@ -17,10 +17,10 @@ import static org.junit.Assert.*; public class JavaExtension { static class TestExtensionId extends AbstractExtensionId implements ExtensionIdProvider { - public final static TestExtensionId instance = new TestExtensionId(); + public final static TestExtensionId TestExtensionProvider = new TestExtensionId(); public ExtensionId lookup() { - return instance; + return TestExtensionId.TestExtensionProvider; } public TestExtension createExtension(ActorSystemImpl i) { @@ -64,8 +64,8 @@ public class JavaExtension { @Test public void mustBeAccessible() { - assertSame(system.extension(TestExtensionId.instance).system, system); - assertSame(TestExtensionId.instance.apply(system).system, system); + assertSame(system.extension(TestExtensionId.TestExtensionProvider).system, system); + assertSame(TestExtensionId.TestExtensionProvider.apply(system).system, system); } @Test diff --git a/akka-docs/java/code/akka/docs/extension/ExtensionDocTestBase.java b/akka-docs/java/code/akka/docs/extension/ExtensionDocTestBase.java index c21f6b7401..6d62eb5bb8 100644 --- a/akka-docs/java/code/akka/docs/extension/ExtensionDocTestBase.java +++ b/akka-docs/java/code/akka/docs/extension/ExtensionDocTestBase.java @@ -30,14 +30,14 @@ public class ExtensionDocTestBase { //#extensionid public static class CountExtension extends AbstractExtensionId implements ExtensionIdProvider { //This will be the identifier of our CountExtension - public final static CountExtension instance = new CountExtension(); + public final static CountExtension CountExtensionProvider = new CountExtension(); //The lookup method is required by ExtensionIdProvider, // so we return ourselves here, this allows us // to configure our extension to be loaded when // the ActorSystem starts up public CountExtension lookup() { - return CountExtension.instance; //The public static final + return CountExtension.CountExtensionProvider; //The public static final } //This method will be called by Akka @@ -52,7 +52,8 @@ public class ExtensionDocTestBase { //#extension-usage-actor public static class MyActor extends UntypedActor { public void onReceive(Object msg) { - CountExtension.instance.get(getContext().system()).increment(); + // typically you would use static import of CountExtension.CountExtensionProvider field + CountExtension.CountExtensionProvider.get(getContext().system()).increment(); } } @@ -63,7 +64,8 @@ public class ExtensionDocTestBase { final ActorSystem system = null; try { //#extension-usage - CountExtension.instance.get(system).increment(); + // typically you would use static import of CountExtension.CountExtensionProvider field + CountExtension.CountExtensionProvider.get(system).increment(); //#extension-usage } catch (Exception e) { //do nothing diff --git a/akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTestBase.java b/akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTestBase.java index b0bddba4b2..6ca1c371d9 100644 --- a/akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTestBase.java +++ b/akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTestBase.java @@ -38,10 +38,10 @@ public class SettingsExtensionDocTestBase { //#extensionid public static class Settings extends AbstractExtensionId implements ExtensionIdProvider { - public final static Settings instance = new Settings(); + public final static Settings SettingsProvider = new Settings(); public Settings lookup() { - return Settings.instance; + return Settings.SettingsProvider; } public SettingsImpl createExtension(ActorSystemImpl system) { @@ -53,7 +53,8 @@ public class SettingsExtensionDocTestBase { //#extension-usage-actor public static class MyActor extends UntypedActor { - final SettingsImpl settings = Settings.instance.get(getContext().system()); + // typically you would use static import of CountExtension.CountExtensionProvider field + final SettingsImpl settings = Settings.SettingsProvider.get(getContext().system()); Connection connection = connect(settings.DB_URI, settings.CIRCUIT_BREAKER_TIMEOUT); //#extension-usage-actor @@ -75,7 +76,8 @@ public class SettingsExtensionDocTestBase { final ActorSystem system = null; try { //#extension-usage - String dbUri = Settings.instance.get(system).DB_URI; + // typically you would use static import of CountExtension.CountExtensionProvider field + String dbUri = Settings.SettingsProvider.get(system).DB_URI; //#extension-usage } catch (Exception e) { //do nothing From 1f87a0326864ddaa5855c6c3d43228b2bce55f2c Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 23 Jan 2012 16:26:10 +0100 Subject: [PATCH 104/152] Linking the 0MQ docs --- akka-docs/scala/index.rst | 1 + 1 file changed, 1 insertion(+) diff --git a/akka-docs/scala/index.rst b/akka-docs/scala/index.rst index 268fc06981..46a84fe064 100644 --- a/akka-docs/scala/index.rst +++ b/akka-docs/scala/index.rst @@ -25,3 +25,4 @@ Scala API io testing extending-akka + zeromq From dbe3a01e9c813fc70b1dd66fb64190728d4c57fd Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 23 Jan 2012 16:53:09 +0100 Subject: [PATCH 105/152] Fixing the 0MQ tests and removing boilerplate --- .../zeromq/ConcurrentSocketActorSpec.scala | 36 ++----------------- 1 file changed, 2 insertions(+), 34 deletions(-) diff --git a/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala b/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala index 4b3c5b46b6..983a2951c1 100644 --- a/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala +++ b/akka-zeromq/src/test/scala/akka/zeromq/ConcurrentSocketActorSpec.scala @@ -5,16 +5,13 @@ package akka.zeromq import org.scalatest.matchers.MustMatchers import akka.testkit.{ TestProbe, DefaultTimeout, AkkaSpec } -import akka.util.Timeout import akka.util.duration._ -import java.net.{ SocketException, ConnectException, Socket } -import util.Random import akka.actor.{ Cancellable, Actor, Props, ActorRef } object ConcurrentSocketActorSpec { val config = """ akka { - extensions = ["akka.zeromq.ZeroMQExtension$"] + extensions = [] } """ } @@ -24,7 +21,7 @@ class ConcurrentSocketActorSpec with MustMatchers with DefaultTimeout { - val endpoint = "tcp://127.0.0.1:%s" format FreePort.randomFreePort() + val endpoint = "tcp://127.0.0.1:%s" format { val s = new java.net.ServerSocket(0); try s.getLocalPort finally s.close() } def zmq = system.extension(ZeroMQExtension) @@ -121,33 +118,4 @@ class ConcurrentSocketActorSpec actorRef ! ZMQMessage(payload.getBytes) } } - - object FreePort { - - def isPortFree(port: Int) = { - try { - val socket = new Socket("127.0.0.1", port) - socket.close() - false - } catch { - case e: ConnectException ⇒ true - case e: SocketException if e.getMessage == "Connection reset by peer" ⇒ true - } - } - - private def newPort = Random.nextInt(55365) + 10000 - - def randomFreePort(maxRetries: Int = 50) = { - var count = 0 - var freePort = newPort - while (!isPortFree(freePort)) { - freePort = newPort - count += 1 - if (count >= maxRetries) { - throw new RuntimeException("Couldn't determine a free port") - } - } - freePort - } - } } From 83ca0cee65ee862279bfd4e1aae612c3a1ae95e5 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Mon, 23 Jan 2012 09:11:18 -0700 Subject: [PATCH 106/152] Make IO tests a bit less abusive --- .../src/test/scala/akka/actor/IOActor.scala | 21 ++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala index 676bb2c4a8..e01729d632 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala @@ -236,7 +236,7 @@ class IOActorSpec extends AkkaSpec with DefaultTimeout { * @param filter determines which exceptions should be retried * @return a future containing the result or the last exception before a limit was hit. */ - def retry[T](count: Option[Int] = None, timeout: Option[Duration] = None, delay: Option[Duration] = None, filter: Option[Throwable ⇒ Boolean] = None)(future: ⇒ Future[T])(implicit executor: ExecutionContext): Future[T] = { + def retry[T](count: Option[Int] = None, timeout: Option[Duration] = None, delay: Option[Duration] = Some(100 millis), filter: Option[Throwable ⇒ Boolean] = None)(future: ⇒ Future[T])(implicit executor: ExecutionContext): Future[T] = { val promise = Promise[T]()(executor) @@ -269,29 +269,34 @@ class IOActorSpec extends AkkaSpec with DefaultTimeout { "an IO Actor" must { "run echo server" in { filterException[java.net.ConnectException] { + val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8064))) val client = system.actorOf(Props(new SimpleEchoClient("localhost", 8064))) val f1 = retry() { client ? ByteString("Hello World!1") } val f2 = retry() { client ? ByteString("Hello World!2") } val f3 = retry() { client ? ByteString("Hello World!3") } - val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8064))) Await.result(f1, TestLatch.DefaultTimeout) must equal(ByteString("Hello World!1")) Await.result(f2, TestLatch.DefaultTimeout) must equal(ByteString("Hello World!2")) Await.result(f3, TestLatch.DefaultTimeout) must equal(ByteString("Hello World!3")) + system.stop(client) + system.stop(server) } } "run echo server under high load" in { filterException[java.net.ConnectException] { - val client = system.actorOf(Props(new SimpleEchoClient("localhost", 8065))) - val list = List.range(0, 1000) - val f = Future.traverse(list)(i ⇒ retry() { client ? ByteString(i.toString) }) val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8065))) - assert(Await.result(f, TestLatch.DefaultTimeout).size === 1000) + val client = system.actorOf(Props(new SimpleEchoClient("localhost", 8065))) + val list = List.range(0, 100) + val f = Future.traverse(list)(i ⇒ retry() { client ? ByteString(i.toString) }) + assert(Await.result(f, TestLatch.DefaultTimeout).size === 100) + system.stop(client) + system.stop(server) } } "run key-value store" in { filterException[java.net.ConnectException] { + val server = system.actorOf(Props(new KVStore("localhost", 8067))) val client1 = system.actorOf(Props(new KVClient("localhost", 8067))) val client2 = system.actorOf(Props(new KVClient("localhost", 8067))) val f1 = retry() { client1 ? KVSet("hello", "World") } @@ -300,13 +305,15 @@ class IOActorSpec extends AkkaSpec with DefaultTimeout { val f4 = f2 flatMap { _ ⇒ retry() { client2 ? KVSet("test", "I'm a test!") } } val f5 = f4 flatMap { _ ⇒ retry() { client1 ? KVGet("test") } } val f6 = Future.sequence(List(f3, f5)) flatMap { _ ⇒ retry() { client2 ? KVGetAll } } - val server = system.actorOf(Props(new KVStore("localhost", 8067))) Await.result(f1, TestLatch.DefaultTimeout) must equal("OK") Await.result(f2, TestLatch.DefaultTimeout) must equal("OK") Await.result(f3, TestLatch.DefaultTimeout) must equal(Some("World")) Await.result(f4, TestLatch.DefaultTimeout) must equal("OK") Await.result(f5, TestLatch.DefaultTimeout) must equal(Some("I'm a test!")) Await.result(f6, TestLatch.DefaultTimeout) must equal(Map("hello" -> "World", "test" -> "I'm a test!")) + system.stop(client1) + system.stop(client2) + system.stop(server) } } } From 9e15d2062bbfa9b0565fe3fcb0bd921b31cf7f64 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Mon, 23 Jan 2012 17:18:49 +0100 Subject: [PATCH 107/152] Renamed more things to SupervisorStrategy. Updated more docs. See #1711 --- .../scala/akka/actor/DeathWatchSpec.scala | 4 +- .../test/scala/akka/actor/Supervisor.scala | 2 +- .../akka/actor/SupervisorHierarchySpec.scala | 6 +- .../scala/akka/actor/TypedActorSpec.scala | 2 +- .../src/main/scala/akka/actor/Actor.scala | 2 +- .../scala/akka/actor/ActorRefProvider.scala | 6 +- .../main/scala/akka/actor/FaultHandling.scala | 44 ++--- .../main/scala/akka/actor/TypedActor.scala | 10 +- .../main/scala/akka/actor/UntypedActor.scala | 6 +- akka-docs/general/actors.rst | 154 +++++++++--------- .../docs/actor/FaultHandlingTestBase.java | 14 +- akka-docs/java/fault-tolerance.rst | 8 +- akka-docs/java/typed-actors.rst | 11 +- akka-docs/java/untyped-actors.rst | 1 + .../project/migration-guide-1.3.x-2.0.x.rst | 20 ++- akka-docs/scala/actors.rst | 1 + .../docs/actor/FaultHandlingDocSpec.scala | 4 +- akka-docs/scala/fault-tolerance.rst | 8 +- akka-docs/scala/typed-actors.rst | 11 +- 19 files changed, 166 insertions(+), 148 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala index 78fd2a6780..781b227f6c 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala @@ -23,7 +23,7 @@ trait DeathWatchSpec { this: AkkaSpec with ImplicitSender with DefaultTimeout import DeathWatchSpec._ - lazy val supervisor = system.actorOf(Props(new Supervisor(FaultHandlingStrategy.defaultFaultHandler)), "watchers") + lazy val supervisor = system.actorOf(Props(new Supervisor(SupervisorStrategy.defaultStrategy)), "watchers") def startWatching(target: ActorRef) = Await.result((supervisor ? props(target, testActor)).mapTo[ActorRef], 3 seconds) @@ -115,7 +115,7 @@ trait DeathWatchSpec { this: AkkaSpec with ImplicitSender with DefaultTimeout "fail a monitor which does not handle Terminated()" in { filterEvents(EventFilter[ActorKilledException](), EventFilter[DeathPactException]()) { case class FF(fail: Failed) - val strategy = new OneForOneStrategy(FaultHandlingStrategy.makeDecider(List(classOf[Exception])), Some(0)) { + val strategy = new OneForOneStrategy(SupervisorStrategy.makeDecider(List(classOf[Exception])), Some(0)) { override def handleFailure(context: ActorContext, child: ActorRef, cause: Throwable, stats: ChildRestartStats, children: Iterable[ChildRestartStats]) = { testActor.tell(FF(Failed(cause)), child) super.handleFailure(context, child, cause, stats, children) diff --git a/akka-actor-tests/src/test/scala/akka/actor/Supervisor.scala b/akka-actor-tests/src/test/scala/akka/actor/Supervisor.scala index d04990c375..80524ea9dc 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/Supervisor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/Supervisor.scala @@ -7,7 +7,7 @@ package akka.actor * For testing Supervisor behavior, normally you don't supply the strategy * from the outside like this. */ -class Supervisor(override val supervisorStrategy: FaultHandlingStrategy) extends Actor { +class Supervisor(override val supervisorStrategy: SupervisorStrategy) extends Actor { def receive = { case x: Props ⇒ sender ! context.actorOf(x) diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala index 2fc5996b54..d0be683c94 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala @@ -16,7 +16,7 @@ object SupervisorHierarchySpec { * For testing Supervisor behavior, normally you don't supply the strategy * from the outside like this. */ - class CountDownActor(countDown: CountDownLatch, override val supervisorStrategy: FaultHandlingStrategy) extends Actor { + class CountDownActor(countDown: CountDownLatch, override val supervisorStrategy: SupervisorStrategy) extends Actor { protected def receive = { case p: Props ⇒ sender ! context.actorOf(p) @@ -43,7 +43,7 @@ class SupervisorHierarchySpec extends AkkaSpec with DefaultTimeout { val managerProps = Props(new CountDownActor(countDown, AllForOneStrategy(List(), None, None))) val manager = Await.result((boss ? managerProps).mapTo[ActorRef], timeout.duration) - val workerProps = Props(new CountDownActor(countDown, FaultHandlingStrategy.defaultFaultHandler)) + val workerProps = Props(new CountDownActor(countDown, SupervisorStrategy.defaultStrategy)) val workerOne, workerTwo, workerThree = Await.result((manager ? workerProps).mapTo[ActorRef], timeout.duration) filterException[ActorKilledException] { @@ -62,7 +62,7 @@ class SupervisorHierarchySpec extends AkkaSpec with DefaultTimeout { val boss = system.actorOf(Props(new Actor { override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), 1, 5000) - val crasher = context.watch(context.actorOf(Props(new CountDownActor(countDownMessages, FaultHandlingStrategy.defaultFaultHandler)))) + val crasher = context.watch(context.actorOf(Props(new CountDownActor(countDownMessages, SupervisorStrategy.defaultStrategy)))) protected def receive = { case "killCrasher" ⇒ crasher ! Kill diff --git a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala index 6ef7608ce2..92cec0a79f 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala @@ -300,7 +300,7 @@ class TypedActorSpec extends AkkaSpec(TypedActorSpec.config) filterEvents(EventFilter[IllegalStateException]("expected")) { val boss = system.actorOf(Props(new Actor { override val supervisorStrategy = OneForOneStrategy { - case e: IllegalStateException if e.getMessage == "expected" ⇒ FaultHandlingStrategy.Resume + case e: IllegalStateException if e.getMessage == "expected" ⇒ SupervisorStrategy.Resume } def receive = { case p: TypedProps[_] ⇒ context.sender ! TypedActor(context).typedActorOf(p) diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index f48b367e56..40b590aa51 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -236,7 +236,7 @@ trait Actor { * User overridable definition the strategy to use for supervising * child actors. */ - def supervisorStrategy(): FaultHandlingStrategy = FaultHandlingStrategy.defaultFaultHandler + def supervisorStrategy(): SupervisorStrategy = SupervisorStrategy.defaultStrategy /** * User overridable callback. diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 0da5c13e1d..49f241d0c6 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -336,7 +336,7 @@ class LocalActorRefProvider( private class Guardian extends Actor { override val supervisorStrategy = { - import akka.actor.FaultHandlingStrategy._ + import akka.actor.SupervisorStrategy._ OneForOneStrategy { case _: ActorKilledException ⇒ Stop case _: ActorInitializationException ⇒ Stop @@ -376,8 +376,6 @@ class LocalActorRefProvider( override def preRestart(cause: Throwable, msg: Option[Any]) {} } - private val guardianProps = Props(new Guardian) - /* * The problem is that ActorRefs need a reference to the ActorSystem to * provide their service. Hence they cannot be created while the @@ -403,6 +401,8 @@ class LocalActorRefProvider( */ def registerExtraNames(_extras: Map[String, InternalActorRef]): Unit = extraNames ++= _extras + private val guardianProps = Props(new Guardian) + lazy val rootGuardian: InternalActorRef = new LocalActorRef(system, guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, rootPath, true) { object Extra { diff --git a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala index 02c94a34bc..e7ae22f57f 100644 --- a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala @@ -44,7 +44,7 @@ case class ChildRestartStats(val child: ActorRef, var maxNrOfRetriesCount: Int = } } -object FaultHandlingStrategy { +object SupervisorStrategy { sealed trait Action /** @@ -95,7 +95,7 @@ object FaultHandlingStrategy { */ def escalate = Escalate - final val defaultFaultHandler: FaultHandlingStrategy = { + final val defaultStrategy: SupervisorStrategy = { def defaultDecider: Decider = { case _: ActorInitializationException ⇒ Stop case _: ActorKilledException ⇒ Stop @@ -158,9 +158,9 @@ object FaultHandlingStrategy { } } -abstract class FaultHandlingStrategy { +abstract class SupervisorStrategy { - import FaultHandlingStrategy._ + import SupervisorStrategy._ def decider: Decider @@ -200,12 +200,12 @@ abstract class FaultHandlingStrategy { object AllForOneStrategy { def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int): AllForOneStrategy = - new AllForOneStrategy(FaultHandlingStrategy.makeDecider(trapExit), + new AllForOneStrategy(SupervisorStrategy.makeDecider(trapExit), if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange)) def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): AllForOneStrategy = - new AllForOneStrategy(FaultHandlingStrategy.makeDecider(trapExit), maxNrOfRetries, withinTimeRange) + new AllForOneStrategy(SupervisorStrategy.makeDecider(trapExit), maxNrOfRetries, withinTimeRange) def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Option[Int]): AllForOneStrategy = - new AllForOneStrategy(FaultHandlingStrategy.makeDecider(trapExit), maxNrOfRetries, None) + new AllForOneStrategy(SupervisorStrategy.makeDecider(trapExit), maxNrOfRetries, None) } /** @@ -214,22 +214,22 @@ object AllForOneStrategy { * maxNrOfRetries = the number of times an actor is allowed to be restarted * withinTimeRange = millisecond time window for maxNrOfRetries, negative means no window */ -case class AllForOneStrategy(decider: FaultHandlingStrategy.Decider, +case class AllForOneStrategy(decider: SupervisorStrategy.Decider, maxNrOfRetries: Option[Int] = None, - withinTimeRange: Option[Int] = None) extends FaultHandlingStrategy { + withinTimeRange: Option[Int] = None) extends SupervisorStrategy { - def this(decider: FaultHandlingStrategy.JDecider, maxNrOfRetries: Int, withinTimeRange: Int) = - this(FaultHandlingStrategy.makeDecider(decider), + def this(decider: SupervisorStrategy.JDecider, maxNrOfRetries: Int, withinTimeRange: Int) = + this(SupervisorStrategy.makeDecider(decider), if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange)) def this(trapExit: JIterable[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) = - this(FaultHandlingStrategy.makeDecider(trapExit), + this(SupervisorStrategy.makeDecider(trapExit), if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange)) def this(trapExit: Array[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) = - this(FaultHandlingStrategy.makeDecider(trapExit), + this(SupervisorStrategy.makeDecider(trapExit), if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange)) @@ -257,12 +257,12 @@ case class AllForOneStrategy(decider: FaultHandlingStrategy.Decider, object OneForOneStrategy { def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int): OneForOneStrategy = - new OneForOneStrategy(FaultHandlingStrategy.makeDecider(trapExit), + new OneForOneStrategy(SupervisorStrategy.makeDecider(trapExit), if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange)) def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): OneForOneStrategy = - new OneForOneStrategy(FaultHandlingStrategy.makeDecider(trapExit), maxNrOfRetries, withinTimeRange) + new OneForOneStrategy(SupervisorStrategy.makeDecider(trapExit), maxNrOfRetries, withinTimeRange) def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Option[Int]): OneForOneStrategy = - new OneForOneStrategy(FaultHandlingStrategy.makeDecider(trapExit), maxNrOfRetries, None) + new OneForOneStrategy(SupervisorStrategy.makeDecider(trapExit), maxNrOfRetries, None) } /** @@ -271,22 +271,22 @@ object OneForOneStrategy { * maxNrOfRetries = the number of times an actor is allowed to be restarted * withinTimeRange = millisecond time window for maxNrOfRetries, negative means no window */ -case class OneForOneStrategy(decider: FaultHandlingStrategy.Decider, +case class OneForOneStrategy(decider: SupervisorStrategy.Decider, maxNrOfRetries: Option[Int] = None, - withinTimeRange: Option[Int] = None) extends FaultHandlingStrategy { + withinTimeRange: Option[Int] = None) extends SupervisorStrategy { - def this(decider: FaultHandlingStrategy.JDecider, maxNrOfRetries: Int, withinTimeRange: Int) = - this(FaultHandlingStrategy.makeDecider(decider), + def this(decider: SupervisorStrategy.JDecider, maxNrOfRetries: Int, withinTimeRange: Int) = + this(SupervisorStrategy.makeDecider(decider), if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange)) def this(trapExit: JIterable[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) = - this(FaultHandlingStrategy.makeDecider(trapExit), + this(SupervisorStrategy.makeDecider(trapExit), if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange)) def this(trapExit: Array[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) = - this(FaultHandlingStrategy.makeDecider(trapExit), + this(SupervisorStrategy.makeDecider(trapExit), if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange)) diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index 9f81f4c754..3bafe9ba4e 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -218,9 +218,9 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi TypedActor.currentContext set null } - override def supervisorStrategy: FaultHandlingStrategy = me match { - case l: SupervisorStrategy ⇒ l.supervisorStrategy - case _ ⇒ super.supervisorStrategy + override def supervisorStrategy(): SupervisorStrategy = me match { + case l: Supervisor ⇒ l.supervisorStrategy + case _ ⇒ super.supervisorStrategy } override def preStart(): Unit = me match { @@ -283,12 +283,12 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi /** * Mix this into your TypedActor to be able to define supervisor strategy */ - trait SupervisorStrategy { + trait Supervisor { /** * User overridable definition the strategy to use for supervising * child actors. */ - def supervisorStrategy: FaultHandlingStrategy = FaultHandlingStrategy.defaultFaultHandler + def supervisorStrategy(): SupervisorStrategy = SupervisorStrategy.defaultStrategy } /** diff --git a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala index 69cb096e26..ba677b750b 100644 --- a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala @@ -37,7 +37,7 @@ import akka.dispatch.{ MessageDispatcher, Promise } * } * } * - * private static FaultHandlingStrategy strategy = new OneForOneStrategy(new Function() { + * private static SupervisorStrategy strategy = new OneForOneStrategy(new Function() { * @Override * public Action apply(Throwable t) { * if (t instanceof ArithmeticException) { @@ -53,7 +53,7 @@ import akka.dispatch.{ MessageDispatcher, Promise } * }, 10, 60000); * * @Override - * public FaultHandlingStrategy supervisorStrategy() { + * public SupervisorStrategy supervisorStrategy() { * return strategy; * } * @@ -116,7 +116,7 @@ abstract class UntypedActor extends Actor { * User overridable definition the strategy to use for supervising * child actors. */ - override def supervisorStrategy(): FaultHandlingStrategy = super.supervisorStrategy() + override def supervisorStrategy(): SupervisorStrategy = super.supervisorStrategy() /** * User overridable callback. diff --git a/akka-docs/general/actors.rst b/akka-docs/general/actors.rst index 2de931565d..952b1b08e4 100644 --- a/akka-docs/general/actors.rst +++ b/akka-docs/general/actors.rst @@ -3,123 +3,121 @@ What is an Actor? ================= -The previous section about :ref:`actor-systems` explained how actors form -hierarchies and are the smallest unit when building an application. This -section looks at one such actor in isolation, explaining the concepts you -encounter while implementing it. For more an in depth reference with all the +The previous section about :ref:`actor-systems` explained how actors form +hierarchies and are the smallest unit when building an application. This +section looks at one such actor in isolation, explaining the concepts you +encounter while implementing it. For more an in depth reference with all the details please refer to :ref:`actors-scala` and :ref:`untyped-actors-java`. -An actor is a container for `State`_, `Behavior`_, a `Mailbox`_, `Children`_ -and a `Fault Handling Strategy`_. All of this is encapsulated behind an `Actor +An actor is a container for `State`_, `Behavior`_, a `Mailbox`_, `Children`_ +and a `Supervisor Strategy`_. All of this is encapsulated behind an `Actor Reference`_. Finally, this happens `When an Actor Terminates`_. Actor Reference --------------- -As detailed below, an actor object needs to be shielded from the outside in -order to benefit from the actor model. Therefore, actors are represented to the -outside using actor references, which are objects that can be passed around -freely and without restriction. This split into inner and outer object enables -transparency for all the desired operations: restarting an actor without -needing to update references elsewhere, placing the actual actor object on -remote hosts, sending messages to actors in completely different applications. -But the most important aspect is that it is not possible to look inside an -actor and get hold of its state from the outside, unless the actor unwisely +As detailed below, an actor object needs to be shielded from the outside in +order to benefit from the actor model. Therefore, actors are represented to the +outside using actor references, which are objects that can be passed around +freely and without restriction. This split into inner and outer object enables +transparency for all the desired operations: restarting an actor without +needing to update references elsewhere, placing the actual actor object on +remote hosts, sending messages to actors in completely different applications. +But the most important aspect is that it is not possible to look inside an +actor and get hold of its state from the outside, unless the actor unwisely publishes this information itself. State ----- -Actor objects will typically contain some variables which reflect possible -states the actor may be in. This can be an explicit state machine (e.g. using -the :ref:`fsm` module), or it could be a counter, set of listeners, pending -requests, etc. These data are what make an actor valuable, and they must be -protected from corruption by other actors. The good news is that Akka actors -conceptually each have their own light-weight thread, which is completely -shielded from the rest of the system. This means that instead of having to -synchronize access using locks you can just write your actor code without +Actor objects will typically contain some variables which reflect possible +states the actor may be in. This can be an explicit state machine (e.g. using +the :ref:`fsm` module), or it could be a counter, set of listeners, pending +requests, etc. These data are what make an actor valuable, and they must be +protected from corruption by other actors. The good news is that Akka actors +conceptually each have their own light-weight thread, which is completely +shielded from the rest of the system. This means that instead of having to +synchronize access using locks you can just write your actor code without worrying about concurrency at all. -Behind the scenes Akka will run sets of actors on sets of real threads, where -typically many actors share one thread, and subsequent invocations of one actor -may end up being processed on different threads. Akka ensures that this -implementation detail does not affect the single-threadedness of handling the +Behind the scenes Akka will run sets of actors on sets of real threads, where +typically many actors share one thread, and subsequent invocations of one actor +may end up being processed on different threads. Akka ensures that this +implementation detail does not affect the single-threadedness of handling the actor’s state. -Because the internal state is vital to an actor’s operations, having -inconsistent state is fatal. Thus, when the actor fails and is restarted by its -supervisor, the state will be created from scratch, like upon first creating +Because the internal state is vital to an actor’s operations, having +inconsistent state is fatal. Thus, when the actor fails and is restarted by its +supervisor, the state will be created from scratch, like upon first creating the actor. This is to enable the ability of self-healing of the system. Behavior -------- -Every time a message is processed, it is matched against the current behavior -of the actor. Behavior means a function which defines the actions to be taken -in reaction to the message at that point in time, say forward a request if the -client is authorized, deny it otherwise. This behavior may change over time, -e.g. because different clients obtain authorization over time, or because the -actor may go into an “out-of-service” mode and later come back. These changes -are achieved by either encoding them in state variables which are read from the -behavior logic, or the function itself may be swapped out at runtime, see the -``become`` and ``unbecome`` operations. However, the initial behavior defined -during construction of the actor object is special in the sense that a restart +Every time a message is processed, it is matched against the current behavior +of the actor. Behavior means a function which defines the actions to be taken +in reaction to the message at that point in time, say forward a request if the +client is authorized, deny it otherwise. This behavior may change over time, +e.g. because different clients obtain authorization over time, or because the +actor may go into an “out-of-service” mode and later come back. These changes +are achieved by either encoding them in state variables which are read from the +behavior logic, or the function itself may be swapped out at runtime, see the +``become`` and ``unbecome`` operations. However, the initial behavior defined +during construction of the actor object is special in the sense that a restart of the actor will reset its behavior to this initial one. Mailbox ------- -An actor’s purpose is the processing of messages, and these messages were sent -to the actor from other actors (or from outside the actor system). The piece -which connects sender and receiver is the actor’s mailbox: each actor has -exactly one mailbox to which all senders enqueue their messages. Enqueuing -happens in the time-order of send operations, which means that messages sent -from different actors may not have a defined order at runtime due to the -apparent randomness of distributing actors across threads. Sending multiple -messages to the same target from the same actor, on the other hand, will +An actor’s purpose is the processing of messages, and these messages were sent +to the actor from other actors (or from outside the actor system). The piece +which connects sender and receiver is the actor’s mailbox: each actor has +exactly one mailbox to which all senders enqueue their messages. Enqueuing +happens in the time-order of send operations, which means that messages sent +from different actors may not have a defined order at runtime due to the +apparent randomness of distributing actors across threads. Sending multiple +messages to the same target from the same actor, on the other hand, will enqueue them in the same order. -There are different mailbox implementations to choose from, the default being a -FIFO: the order of the messages processed by the actor matches the order in -which they were enqueued. This is usually a good default, but applications may -need to prioritize some messages over others. In this case, a priority mailbox -will enqueue not always at the end but at a position as given by the message -priority, which might even be at the front. While using such a queue, the order -of messages processed will naturally be defined by the queue’s algorithm and in +There are different mailbox implementations to choose from, the default being a +FIFO: the order of the messages processed by the actor matches the order in +which they were enqueued. This is usually a good default, but applications may +need to prioritize some messages over others. In this case, a priority mailbox +will enqueue not always at the end but at a position as given by the message +priority, which might even be at the front. While using such a queue, the order +of messages processed will naturally be defined by the queue’s algorithm and in general not be FIFO. -An important feature in which Akka differs from some other actor model -implementations is that the current behavior must always handle the next -dequeued message, there is no scanning the mailbox for the next matching one. -Failure to handle a message will typically be treated as a failure, unless this +An important feature in which Akka differs from some other actor model +implementations is that the current behavior must always handle the next +dequeued message, there is no scanning the mailbox for the next matching one. +Failure to handle a message will typically be treated as a failure, unless this behavior is overridden. Children -------- -Each actor is potentially a supervisor: if it creates children for delegating -sub-tasks, it will automatically supervise them. The list of children is -maintained within the actor’s context and the actor has access to it. -Modifications to the list are done by creating (``context.actorOf(...)``) or -stopping (``context.stop(child)``) children and these actions are reflected -immediately. The actual creation and termination actions happen behind the +Each actor is potentially a supervisor: if it creates children for delegating +sub-tasks, it will automatically supervise them. The list of children is +maintained within the actor’s context and the actor has access to it. +Modifications to the list are done by creating (``context.actorOf(...)``) or +stopping (``context.stop(child)``) children and these actions are reflected +immediately. The actual creation and termination actions happen behind the scenes in an asynchronous way, so they do not “block” their supervisor. -Fault Handling Strategy ------------------------ +Supervisor Strategy +------------------- -The final piece of an actor is its strategy for handling faults of its -children. To keep it simple and robust, this is declared outside of the actor’s -code and has no access to the actor’s state. Fault handling is then done -transparently by Akka, applying one of the strategies described in -:ref:`supervision` for each incoming failure. As this strategy is fundamental -to how an actor system is structured, it cannot be changed once an actor has -been created. +The final piece of an actor is its strategy for handling faults of its +children. Fault handling is then done transparently by Akka, applying one +of the strategies described in :ref:`supervision` for each incoming failure. +As this strategy is fundamental to how an actor system is structured, it +cannot be changed once an actor has been created. -Considering that there is only one such strategy for each actor, this means -that if different strategies apply to the various children of an actor, the -children should be grouped beneath intermediate supervisors with matching -strategies, preferring once more the structuring of actor systems according to +Considering that there is only one such strategy for each actor, this means +that if different strategies apply to the various children of an actor, the +children should be grouped beneath intermediate supervisors with matching +strategies, preferring once more the structuring of actor systems according to the splitting of tasks into sub-tasks. When an Actor Terminates diff --git a/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java b/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java index 87610a2aef..2ed0c3d715 100644 --- a/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java +++ b/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java @@ -6,8 +6,8 @@ package akka.docs.actor; //#testkit import akka.actor.ActorRef; import akka.actor.ActorSystem; -import akka.actor.FaultHandlingStrategy; -import static akka.actor.FaultHandlingStrategy.*; +import akka.actor.SupervisorStrategy; +import static akka.actor.SupervisorStrategy.*; import akka.actor.OneForOneStrategy; import akka.actor.Props; import akka.actor.Terminated; @@ -38,7 +38,7 @@ public class FaultHandlingTestBase { static public class Supervisor extends UntypedActor { //#strategy - private static FaultHandlingStrategy strategy = new OneForOneStrategy(new Function() { + private static SupervisorStrategy strategy = new OneForOneStrategy(new Function() { @Override public Action apply(Throwable t) { if (t instanceof ArithmeticException) { @@ -54,7 +54,7 @@ public class FaultHandlingTestBase { }, 10, 60000); @Override - public FaultHandlingStrategy supervisorStrategy() { + public SupervisorStrategy supervisorStrategy() { return strategy; } @@ -75,7 +75,7 @@ public class FaultHandlingTestBase { static public class Supervisor2 extends UntypedActor { //#strategy2 - private static FaultHandlingStrategy strategy = new OneForOneStrategy(new Function() { + private static SupervisorStrategy strategy = new OneForOneStrategy(new Function() { @Override public Action apply(Throwable t) { if (t instanceof ArithmeticException) { @@ -91,7 +91,7 @@ public class FaultHandlingTestBase { }, 10, 60000); @Override - public FaultHandlingStrategy supervisorStrategy() { + public SupervisorStrategy supervisorStrategy() { return strategy; } @@ -147,7 +147,7 @@ public class FaultHandlingTestBase { } @Test - public void mustEmployFaultHandler() { + public void mustEmploySupervisorStrategy() { // code here //#testkit EventFilter ex1 = (EventFilter) new ErrorFilter(ArithmeticException.class); diff --git a/akka-docs/java/fault-tolerance.rst b/akka-docs/java/fault-tolerance.rst index 4d0b1abe8c..1f62f77875 100644 --- a/akka-docs/java/fault-tolerance.rst +++ b/akka-docs/java/fault-tolerance.rst @@ -1,7 +1,7 @@ .. _fault-tolerance-java: -Fault Handling Strategies (Java) -================================= +Fault Tolerance (Java) +====================== .. sidebar:: Contents @@ -12,8 +12,8 @@ children, and as such each actor defines fault handling supervisor strategy. This strategy cannot be changed afterwards as it is an integral part of the actor system’s structure. -Creating a Fault Handling Strategy ----------------------------------- +Creating a Supervisor Strategy +------------------------------ For the sake of demonstration let us consider the following strategy: diff --git a/akka-docs/java/typed-actors.rst b/akka-docs/java/typed-actors.rst index 4b7de5ee09..d6b016e3b5 100644 --- a/akka-docs/java/typed-actors.rst +++ b/akka-docs/java/typed-actors.rst @@ -30,7 +30,7 @@ it's located in ``akka.actor.TypedActor``. :include: typed-actor-extension-tools .. warning:: - + Same as not exposing ``this`` of an Akka Actor, it's important not to expose ``this`` of a Typed Actor, instead you should pass the external proxy reference, which is obtained from within your Typed Actor as ``TypedActor.self()``, this is your external identity, as the ``ActorRef`` is the external identity of @@ -127,7 +127,7 @@ Request-reply-with-future message send .. includecode:: code/akka/docs/actor/TypedActorDocTestBase.java :include: typed-actor-call-future -This call is asynchronous, and the Future returned can be used for asynchronous composition. +This call is asynchronous, and the Future returned can be used for asynchronous composition. Stopping Typed Actors --------------------- @@ -153,6 +153,13 @@ you can create child Typed Actors by invoking ``typedActorOf(..)`` on that. This also works for creating child Typed Actors in regular Akka Actors. +Supervisor Strategy +------------------- + +By having your Typed Actor implementation class implement ``TypedActor.Supervisor`` +you can define the strategy to use for supervising child actors, as described in +:ref:`supervision` and :ref:`fault-tolerance-java`. + Lifecycle callbacks ------------------- diff --git a/akka-docs/java/untyped-actors.rst b/akka-docs/java/untyped-actors.rst index 7e0d788590..fa8b83db71 100644 --- a/akka-docs/java/untyped-actors.rst +++ b/akka-docs/java/untyped-actors.rst @@ -129,6 +129,7 @@ In addition, it offers: * :obj:`getSelf()` reference to the :class:`ActorRef` of the actor * :obj:`getSender()` reference sender Actor of the last received message, typically used as described in :ref:`UntypedActor.Reply` +* :obj:`supervisorStrategy()` user overridable definition the strategy to use for supervising child actors * :obj:`getContext()` exposes contextual information for the actor and the current message, such as: * factory methods to create child actors (:meth:`actorOf`) diff --git a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst index a3f874894d..6199105e0f 100644 --- a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst +++ b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst @@ -403,7 +403,7 @@ v2.0:: context.parent -*Fault handling strategy* +*Supervisor Strategy* v1.3:: @@ -420,14 +420,18 @@ v1.3:: v2.0:: - val strategy = OneForOneStrategy({ - case _: ArithmeticException ⇒ Resume - case _: NullPointerException ⇒ Restart - case _: IllegalArgumentException ⇒ Stop - case _: Exception ⇒ Escalate - }: Decider, maxNrOfRetries = Some(10), withinTimeRange = Some(60000)) + class MyActor extends Actor { + override val supervisorStrategy = OneForOneStrategy({ + case _: ArithmeticException ⇒ Resume + case _: NullPointerException ⇒ Restart + case _: IllegalArgumentException ⇒ Stop + case _: Exception ⇒ Escalate + }: Decider, maxNrOfRetries = Some(10), withinTimeRange = Some(60000)) - val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(strategy), "supervisor") + def receive = { + case x => + } + } Documentation: diff --git a/akka-docs/scala/actors.rst b/akka-docs/scala/actors.rst index 558b50fac8..61b49c856f 100644 --- a/akka-docs/scala/actors.rst +++ b/akka-docs/scala/actors.rst @@ -154,6 +154,7 @@ In addition, it offers: * :obj:`self` reference to the :class:`ActorRef` of the actor * :obj:`sender` reference sender Actor of the last received message, typically used as described in :ref:`Actor.Reply` +* :obj:`supervisorStrategy` user overridable definition the strategy to use for supervising child actors * :obj:`context` exposes contextual information for the actor and the current message, such as: * factory methods to create child actors (:meth:`actorOf`) diff --git a/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala b/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala index dc9159de43..22790d2110 100644 --- a/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala @@ -19,7 +19,7 @@ object FaultHandlingDocSpec { class Supervisor extends Actor { //#strategy import akka.actor.OneForOneStrategy - import akka.actor.FaultHandlingStrategy._ + import akka.actor.SupervisorStrategy._ override val supervisorStrategy = OneForOneStrategy({ case _: ArithmeticException ⇒ Resume @@ -39,7 +39,7 @@ object FaultHandlingDocSpec { class Supervisor2 extends Actor { //#strategy2 import akka.actor.OneForOneStrategy - import akka.actor.FaultHandlingStrategy._ + import akka.actor.SupervisorStrategy._ override val supervisorStrategy = OneForOneStrategy({ case _: ArithmeticException ⇒ Resume diff --git a/akka-docs/scala/fault-tolerance.rst b/akka-docs/scala/fault-tolerance.rst index a4cdd584f0..f17cab2bb3 100644 --- a/akka-docs/scala/fault-tolerance.rst +++ b/akka-docs/scala/fault-tolerance.rst @@ -1,7 +1,7 @@ .. _fault-tolerance-scala: -Fault Handling Strategies (Scala) -================================= +Fault Tolerance (Scala) +======================= .. sidebar:: Contents @@ -12,8 +12,8 @@ children, and as such each actor defines fault handling supervisor strategy. This strategy cannot be changed afterwards as it is an integral part of the actor system’s structure. -Creating a Fault Handling Strategy ----------------------------------- +Creating a Supervisor Strategy +------------------------------ For the sake of demonstration let us consider the following strategy: diff --git a/akka-docs/scala/typed-actors.rst b/akka-docs/scala/typed-actors.rst index 7a5a87aeb4..ba326b8a27 100644 --- a/akka-docs/scala/typed-actors.rst +++ b/akka-docs/scala/typed-actors.rst @@ -30,7 +30,7 @@ it's located in ``akka.actor.TypedActor``. :include: typed-actor-extension-tools .. warning:: - + Same as not exposing ``this`` of an Akka Actor, it's important not to expose ``this`` of a Typed Actor, instead you should pass the external proxy reference, which is obtained from within your Typed Actor as ``TypedActor.self``, this is your external identity, as the ``ActorRef`` is the external identity of @@ -127,7 +127,7 @@ Request-reply-with-future message send .. includecode:: code/akka/docs/actor/TypedActorDocSpec.scala :include: typed-actor-call-future -This call is asynchronous, and the Future returned can be used for asynchronous composition. +This call is asynchronous, and the Future returned can be used for asynchronous composition. Stopping Typed Actors --------------------- @@ -153,6 +153,13 @@ you can create child Typed Actors by invoking ``typedActorOf(..)`` on that. This also works for creating child Typed Actors in regular Akka Actors. +Supervisor Strategy +------------------- + +By having your Typed Actor implementation class implement ``TypedActor.Supervisor`` +you can define the strategy to use for supervising child actors, as described in +:ref:`supervision` and :ref:`fault-tolerance-scala`. + Lifecycle callbacks ------------------- From e087c6fd8cae702936612a815f35641a3c40a406 Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 23 Jan 2012 17:43:30 +0100 Subject: [PATCH 108/152] include FSMDocSpec example in fsm.rst and fix a few outdated things --- .../code/akka/docs/actor/FSMDocSpec.scala | 23 ++- akka-docs/scala/fsm.rst | 189 +++++++----------- 2 files changed, 82 insertions(+), 130 deletions(-) diff --git a/akka-docs/scala/code/akka/docs/actor/FSMDocSpec.scala b/akka-docs/scala/code/akka/docs/actor/FSMDocSpec.scala index 5668d88868..684d0eea3e 100644 --- a/akka-docs/scala/code/akka/docs/actor/FSMDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/actor/FSMDocSpec.scala @@ -3,13 +3,16 @@ */ package akka.docs.actor +//#test-code import akka.testkit.AkkaSpec +import akka.actor.Props class FSMDocSpec extends AkkaSpec { "simple finite state machine" must { + //#fsm-code-elided //#simple-imports - import akka.actor.{ Actor, ActorRef, FSM, Props } + import akka.actor.{ Actor, ActorRef, FSM } import akka.util.duration._ //#simple-imports //#simple-events @@ -23,11 +26,11 @@ class FSMDocSpec extends AkkaSpec { //#simple-events //#simple-state // states - trait State + sealed trait State case object Idle extends State case object Active extends State - trait Data + sealed trait Data case object Uninitialized extends Data case class Todo(target: ActorRef, queue: Seq[Any]) extends Data //#simple-state @@ -40,20 +43,20 @@ class FSMDocSpec extends AkkaSpec { case Event(SetTarget(ref), Uninitialized) ⇒ stay using Todo(ref, Vector.empty) } - //#simple-transition + //#transition-elided onTransition { case Active -> Idle ⇒ stateData match { case Todo(ref, queue) ⇒ ref ! Batch(queue) } } - //#simple-transition + //#transition-elided when(Active, stateTimeout = 1 second) { case Event(Flush | FSM.StateTimeout, t: Todo) ⇒ goto(Idle) using t.copy(queue = Vector.empty) } - //#simple-unhandled + //#unhandled-elided whenUnhandled { // common code for both states case Event(Queue(obj), t @ Todo(_, v)) ⇒ @@ -63,11 +66,12 @@ class FSMDocSpec extends AkkaSpec { log.warning("received unhandled request {} in state {}/{}", e, stateName, s) stay } - //#simple-unhandled + //#unhandled-elided initialize } //#simple-fsm + //#fsm-code-elided "batch correctly" in { val buncher = system.actorOf(Props(new Buncher)) @@ -87,7 +91,6 @@ class FSMDocSpec extends AkkaSpec { buncher ! Queue(42) expectNoMsg } - } - -} \ No newline at end of file +} +//#test-code diff --git a/akka-docs/scala/fsm.rst b/akka-docs/scala/fsm.rst index af2b36ad4c..7b3d136ae4 100644 --- a/akka-docs/scala/fsm.rst +++ b/akka-docs/scala/fsm.rst @@ -26,146 +26,104 @@ These relations are interpreted as meaning: A Simple Example ================ -To demonstrate the usage of states we start with a simple FSM without state -data. The state can be of any type so for this example we create the states A, -B and C. +To demonstrate most of the features of the :class:`FSM` trait, consider an +actor which shall receive and queue messages while they arrive in a burst and +send them on after the burst ended or a flush request is received. -.. code-block:: scala +First, consider all of the below to use these import statements: - sealed trait ExampleState - case object A extends ExampleState - case object B extends ExampleState - case object C extends ExampleState +.. includecode:: code/akka/docs/actor/FSMDocSpec.scala#simple-imports -Now lets create an object representing the FSM and defining the behavior. +The contract of our “Buncher” actor is that is accepts or produces the following messages: -.. code-block:: scala +.. includecode:: code/akka/docs/actor/FSMDocSpec.scala#simple-events - import akka.actor.{Actor, FSM} - import akka.util.duration._ +``SetTarget`` is needed for starting it up, setting the destination for the +``Batches`` to be passed on; ``Queue`` will add to the internal queue while +``Flush`` will mark the end of a burst. - case object Move +.. includecode:: code/akka/docs/actor/FSMDocSpec.scala#simple-state - class ABC extends Actor with FSM[ExampleState, Unit] { +The actor can be in two states: no message queued (aka ``Idle``) or some +message queued (aka ``Active``). It will stay in the active state as long as +messages keep arriving and no flush is requested. The internal state data of +the actor is made up of the target actor reference to send the batches to and +the actual queue of messages. - import FSM._ +Now let’s take a look at the skeleton for our FSM actor: - startWith(A, Unit) +.. includecode:: code/akka/docs/actor/FSMDocSpec.scala + :include: simple-fsm + :exclude: transition-elided,unhandled-elided - when(A) { - case Ev(Move) => - log.info(this, "Go to B and move on after 5 seconds") - goto(B) forMax (5 seconds) - } +The basic strategy is to declare the actor, mixing in the :class:`FSM` trait +and specifying the possible states and data values as type paramters. Within +the body of the actor a DSL is used for declaring the state machine: - when(B) { - case Ev(StateTimeout) => - log.info(this, "Moving to C") - goto(C) - } + * :meth:`startsWith` defines the initial state and initial data + * then there is one :meth:`when() { ... }` declaration per state to be + handled (could potentially be multiple ones, the passed + :class:`PartialFunction` will be concatenated using :meth:`orElse`) + * finally starting it up using :meth:`initialize`, which performs the + transition into the initial state and sets up timers (if required). - when(C) { - case Ev(Move) => - log.info(this, "Stopping") - stop - } +In this case, we start out in the ``Idle`` and ``Uninitialized`` state, where +only the ``SetTarget()`` message is handled; ``stay`` prepares to end this +event’s processing for not leaving the current state, while the ``using`` +modifier makes the FSM replace the internal state (which is ``Uninitialized`` +at this point) with a fresh ``Todo()`` object containing the target actor +reference. The ``Active`` state has a state timeout declared, which means that +if no message is received for 1 second, a ``FSM.StateTimeout`` message will be +generated. This has the same effect as receiving the ``Flush`` command in this +case, namely to transition back into the ``Idle`` state and resetting the +internal queue to the empty vector. But how do messages get queued? Since this +shall work identically in both states, we make use of the fact that any event +which is not handled by the ``when()`` block is passed to the +``whenUnhandled()`` block: - initialize // this checks validity of the initial state and sets up timeout if needed - } +.. includecode:: code/akka/docs/actor/FSMDocSpec.scala#unhandled-elided -Each state is described by one or more :func:`when(state)` blocks; if more than -one is given for the same state, they are tried in the order given until the -first is found which matches the incoming event. Events are matched using -either :func:`Ev(msg)` (if no state data are to be extracted) or -:func:`Event(msg, data)`, see below. The statements for each case are the -actions to be taken, where the final expression must describe the transition -into the next state. This can either be :func:`stay` when no transition is -needed or :func:`goto(target)` for changing into the target state. The -transition may be annotated with additional properties, where this example -includes a state timeout of 5 seconds after the transition into state B: -:func:`forMax(duration)` arranges for a :obj:`StateTimeout` message to be -scheduled, unless some other message is received first. The construction of the -FSM is finished by calling the :func:`initialize` method as last part of the -ABC constructor. +The first case handled here is adding ``Queue()`` requests to the internal +queue and going to the ``Active`` state (this does the obvious thing of staying +in the ``Active`` state if already there), but only if the FSM data are not +``Uninitialized`` when the ``Queue()`` event is received. Otherwise—and in all +other non-handled cases—the second case just logs a warning and does not change +the internal state. -State Data -========== +The only missing piece is where the ``Batches`` are actually sent to the +target, for which we use the ``onTransition`` mechanism: you can declare +multiple such blocks and all of them will be tried for matching behavior in +case a state transition occurs (i.e. only when the state actually changes). -The FSM can also hold state data associated with the internal state of the -state machine. The state data can be of any type but to demonstrate let's look -at a lock with a :class:`String` as state data holding the entered unlock code. -First we need two states for the lock: +.. includecode:: code/akka/docs/actor/FSMDocSpec.scala#transition-elided -.. code-block:: scala +The transition callback is a partial function which takes as input a pair of +states—the current and the next state. The FSM trait includes a convenience +extractor for these in form of an arrow operator, which conveniently reminds +you of the direction of the state change which is being matched. During the +state change, the old state data is available via ``stateData`` as shown, and +the new state data would be available as ``nextStateData``. - sealed trait LockState - case object Locked extends LockState - case object Open extends LockState +To verify that this buncher actually works, it is quite easy to write a test +using the :ref:`akka-testkit`, which is conveniently bundled with ScalaTest traits +into ``AkkaSpec``: -Now we can create a lock FSM that takes :class:`LockState` as a state and a -:class:`String` as state data: - -.. code-block:: scala - - import akka.actor.{Actor, FSM} - - class Lock(code: String) extends Actor with FSM[LockState, String] { - - import FSM._ - - val emptyCode = "" - - startWith(Locked, emptyCode) - - when(Locked) { - // receive a digit and the code that we have so far - case Event(digit: Char, soFar) => { - // add the digit to what we have - soFar + digit match { - case incomplete if incomplete.length < code.length => - // not enough digits yet so stay using the - // incomplete code as the new state data - stay using incomplete - case `code` => - // code matched the one from the lock - // so go to Open state and reset the state data - goto(Open) using emptyCode forMax (1 seconds) - case wrong => - // wrong code, stay Locked and reset the state data - stay using emptyCode - } - } - } - - when(Open) { - case Ev(StateTimeout, _) => { - // after the timeout, go back to Locked state - goto(Locked) - } - } - - initialize - } - -This very simple example shows how the complete state of the FSM is encoded in -the :obj:`(State, Data)` pair and only explicitly updated during transitions. -This encapsulation is what makes state machines a powerful abstraction, e.g. -for handling socket states in a network server application. +.. includecode:: code/akka/docs/actor/FSMDocSpec.scala + :include: test-code + :exclude: fsm-code-elided Reference ========= -This section describes the DSL in a more formal way, refer to `Examples`_ for more sample material. - The FSM Trait and Object ------------------------ The :class:`FSM` trait may only be mixed into an :class:`Actor`. Instead of extending :class:`Actor`, the self type approach was chosen in order to make it obvious that an actor is actually created. Importing all members of the -:obj:`FSM` object is recommended to receive useful implicits and directly -access the symbols like :obj:`StateTimeout`. This import is usually placed -inside the state machine definition: +:obj:`FSM` object is recommended if you want to directly access the symbols +like :obj:`StateTimeout`. This import is usually placed inside the state +machine definition: .. code-block:: scala @@ -192,15 +150,6 @@ The :class:`FSM` trait takes two type parameters: to the FSM class you have the advantage of making all changes of the internal state explicit in a few well-known places. -Defining Timeouts ------------------ - -The :class:`FSM` module uses :ref:`Duration` for all timing configuration. -Several methods, like :func:`when()` and :func:`startWith()` take a -:class:`FSM.Timeout`, which is an alias for :class:`Option[Duration]`. There is -an implicit conversion available in the :obj:`FSM` object which makes this -transparent, just import it into your FSM body. - Defining States --------------- From b2c63062fd5e13cfa591c01baa6173af1da01a75 Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 23 Jan 2012 17:56:52 +0100 Subject: [PATCH 109/152] document EmptyLocalActorRef, see #1605 --- akka-docs/general/addressing.rst | 16 ++++++---------- akka-docs/java/untyped-actors.rst | 4 ++++ akka-docs/scala/actors.rst | 4 ++++ 3 files changed, 14 insertions(+), 10 deletions(-) diff --git a/akka-docs/general/addressing.rst b/akka-docs/general/addressing.rst index 86c22fdaea..5a9abc30f3 100644 --- a/akka-docs/general/addressing.rst +++ b/akka-docs/general/addressing.rst @@ -50,12 +50,17 @@ depending on the configuration of the actor system: - There are several special types of actor references which behave like local actor references for all practical purposes: - - :class:`AskActorRef` is the special representation of a :meth:`Promise` for + - :class:`PromiseActorRef` is the special representation of a :meth:`Promise` for the purpose of being completed by the response from an actor; it is created by the :meth:`ActorRef.ask` invocation. - :class:`DeadLetterActorRef` is the default implementation of the dead letters service, where all messages are re-routed whose routees are shut down or non-existent. + - :class:`EmptyLocalActorRef` is what is returned when looking up a + non-existing local actor path: it is equivalent to a + :class:`DeadLetterActorRef`, but it retains its path so that it can be sent + over the network and compared to other existing actor refs for that path, + some of which might have been obtained before the actor stopped existing. - And then there are some one-off internal implementations which you should never really see: @@ -309,12 +314,3 @@ other actors are found. The next level consists of the following: - ``"/remote"`` is an artificial path below which all actors reside whose supervisors are remote actor references -Future extensions: - -- ``"/service"`` is an artificial path below which actors can be presented by - means of configuration, i.e. deployed at system start-up or just-in-time - (triggered by look-up) -- ``"/alias"`` is an artificial path below which other actors may be “mounted” - (as in the Unix file-system sense) by path—local or remote—to give them - logical names. - diff --git a/akka-docs/java/untyped-actors.rst b/akka-docs/java/untyped-actors.rst index 7e0d788590..d6d9bfc8be 100644 --- a/akka-docs/java/untyped-actors.rst +++ b/akka-docs/java/untyped-actors.rst @@ -254,6 +254,10 @@ currently traversed actor, otherwise it will step “down” to the named child. It should be noted that the ``..`` in actor paths here always means the logical structure, i.e. the supervisor. +If the path being looked up does not exist, a special actor reference is +returned which behaves like the actor system’s dead letter queue but retains +its identity (i.e. the path which was looked up). + Remote actor addresses may also be looked up, if remoting is enabled:: getContext().actorFor("akka://app@otherhost:1234/user/serviceB") diff --git a/akka-docs/scala/actors.rst b/akka-docs/scala/actors.rst index 558b50fac8..a2f9965312 100644 --- a/akka-docs/scala/actors.rst +++ b/akka-docs/scala/actors.rst @@ -286,6 +286,10 @@ currently traversed actor, otherwise it will step “down” to the named child. It should be noted that the ``..`` in actor paths here always means the logical structure, i.e. the supervisor. +If the path being looked up does not exist, a special actor reference is +returned which behaves like the actor system’s dead letter queue but retains +its identity (i.e. the path which was looked up). + Remote actor addresses may also be looked up, if remoting is enabled:: context.actorFor("akka://app@otherhost:1234/user/serviceB") From 2557d594cb8f082c96fd9bc15e1c7ee8118b349a Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 23 Jan 2012 18:08:00 +0100 Subject: [PATCH 110/152] Fixing misconception around hotswapped behavior --- akka-docs/java/untyped-actors.rst | 4 ++-- akka-docs/scala/actors.rst | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/akka-docs/java/untyped-actors.rst b/akka-docs/java/untyped-actors.rst index 7e0d788590..1c4cada1d6 100644 --- a/akka-docs/java/untyped-actors.rst +++ b/akka-docs/java/untyped-actors.rst @@ -213,8 +213,8 @@ processing a message. This restart involves the hooks mentioned above: An actor restart replaces only the actual actor object; the contents of the -mailbox and the hotswap stack are unaffected by the restart, so processing of -messages will resume after the :meth:`postRestart` hook returns. The message +mailbox is unaffected by the restart, so processing of messages will resume +after the :meth:`postRestart` hook returns. The message that triggered the exception will not be received again. Any message sent to an actor while it is being restarted will be queued to its mailbox as usual. diff --git a/akka-docs/scala/actors.rst b/akka-docs/scala/actors.rst index 558b50fac8..265fddd064 100644 --- a/akka-docs/scala/actors.rst +++ b/akka-docs/scala/actors.rst @@ -246,8 +246,8 @@ processing a message. This restart involves the hooks mentioned above: An actor restart replaces only the actual actor object; the contents of the -mailbox and the hotswap stack are unaffected by the restart, so processing of -messages will resume after the :meth:`postRestart` hook returns. The message +mailbox is unaffected by the restart, so processing of messages will resume +after the :meth:`postRestart` hook returns. The message that triggered the exception will not be received again. Any message sent to an actor while it is being restarted will be queued to its mailbox as usual. From 7c9b044e11ff0c4769a07ac79aa7cf7e6c00265b Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 23 Jan 2012 18:23:34 +0100 Subject: [PATCH 111/152] =?UTF-8?q?incorporate=20Viktor=E2=80=99s=20review?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../akka/migration/AskableActorRef.scala | 12 +++++++++--- .../src/main/scala/akka/actor/ActorRef.scala | 19 ++++++++++--------- .../scala/akka/actor/ActorRefProvider.scala | 4 ++-- .../src/main/scala/akka/actor/Locker.scala | 5 ++++- .../main/scala/akka/pattern/AskSupport.scala | 12 +++++++++--- 5 files changed, 34 insertions(+), 18 deletions(-) diff --git a/akka-actor-migration/src/main/scala/akka/migration/AskableActorRef.scala b/akka-actor-migration/src/main/scala/akka/migration/AskableActorRef.scala index fc4f28cd8b..942d8ae47a 100644 --- a/akka-actor-migration/src/main/scala/akka/migration/AskableActorRef.scala +++ b/akka-actor-migration/src/main/scala/akka/migration/AskableActorRef.scala @@ -7,7 +7,10 @@ import akka.actor.ActorRef import akka.dispatch.Future import akka.util.Timeout -class AskableActorRef(val actorRef: ActorRef) { +/** + * Implementation detail of the “ask” pattern enrichment of ActorRef + */ +private[akka] final class AskableActorRef(val actorRef: ActorRef) { /** * Sends a message asynchronously and returns a [[akka.dispatch.Future]] @@ -29,8 +32,8 @@ class AskableActorRef(val actorRef: ActorRef) { * Recommended usage: * * {{{ - * val f = worker.ask(request)(timeout) * flow { + * val f = worker.ask(request)(timeout) * EnrichedRequest(request, f()) * } pipeTo nextActor * }}} @@ -59,8 +62,8 @@ class AskableActorRef(val actorRef: ActorRef) { * Recommended usage: * * {{{ - * val f = worker ? request * flow { + * val f = worker ? request * EnrichedRequest(request, f()) * } pipeTo nextActor * }}} @@ -69,6 +72,9 @@ class AskableActorRef(val actorRef: ActorRef) { */ def ?(message: Any)(implicit timeout: Timeout): Future[Any] = akka.pattern.ask(actorRef, message)(timeout) + /** + * This method is just there to catch 2.0-unsupported usage and print deprecation warnings for it. + */ @deprecated("use ?(msg)(timeout), this method has dangerous ambiguity", "2.0-migration") def ?(message: Any, timeout: Timeout)(i: Int = 0): Future[Any] = this.?(message)(timeout) } \ No newline at end of file diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 2ed816a8fb..5ccce6906c 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -355,7 +355,7 @@ case class SerializedActorRef(path: String) { /** * Trait for ActorRef implementations where all methods contain default stubs. */ -trait MinimalActorRef extends InternalActorRef with LocalRef { +private[akka] trait MinimalActorRef extends InternalActorRef with LocalRef { def getParent: InternalActorRef = Nobody @@ -381,7 +381,7 @@ trait MinimalActorRef extends InternalActorRef with LocalRef { protected def writeReplace(): AnyRef = SerializedActorRef(path.toString) } -object MinimalActorRef { +private[akka] object MinimalActorRef { def apply(_path: ActorPath, _provider: ActorRefProvider)(receive: PartialFunction[Any, Unit]): ActorRef = new MinimalActorRef { def path = _path def provider = _provider @@ -392,7 +392,7 @@ object MinimalActorRef { case class DeadLetter(message: Any, sender: ActorRef, recipient: ActorRef) -object DeadLetterActorRef { +private[akka] object DeadLetterActorRef { class SerializedDeadLetterActorRef extends Serializable { //TODO implement as Protobuf for performance? @throws(classOf[java.io.ObjectStreamException]) private def readResolve(): AnyRef = Serialization.currentSystem.value.deadLetters @@ -401,7 +401,7 @@ object DeadLetterActorRef { val serialized = new SerializedDeadLetterActorRef } -trait DeadLetterActorRefLike extends MinimalActorRef { +private[akka] trait DeadLetterActorRefLike extends MinimalActorRef { def eventStream: EventStream @@ -427,11 +427,9 @@ trait DeadLetterActorRefLike extends MinimalActorRef { case d: DeadLetter ⇒ eventStream.publish(d) case _ ⇒ eventStream.publish(DeadLetter(message, sender, this)) } - - // FIXME reimplement behavior of brokenPromise on ask } -class DeadLetterActorRef(val eventStream: EventStream) extends DeadLetterActorRefLike { +private[akka] class DeadLetterActorRef(val eventStream: EventStream) extends DeadLetterActorRefLike { @throws(classOf[java.io.ObjectStreamException]) override protected def writeReplace(): AnyRef = DeadLetterActorRef.serialized } @@ -440,7 +438,7 @@ class DeadLetterActorRef(val eventStream: EventStream) extends DeadLetterActorRe * This special dead letter reference has a name: it is that which is returned * by a local look-up which is unsuccessful. */ -class EmptyLocalActorRef( +private[akka] class EmptyLocalActorRef( val eventStream: EventStream, _provider: ActorRefProvider, _dispatcher: MessageDispatcher, @@ -454,7 +452,10 @@ class EmptyLocalActorRef( } } -class VirtualPathContainer( +/** + * Internal implementation detail used for paths like “/temp” + */ +private[akka] class VirtualPathContainer( val provider: ActorRefProvider, val path: ActorPath, override val getParent: InternalActorRef, diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index f186710e39..da85eda2bf 100755 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -74,12 +74,12 @@ trait ActorRefProvider { /** * Registers an actorRef at a path returned by tempPath(); do NOT pass in any other path. */ - def registerTempActor(actorRef: InternalActorRef, path: ActorPath) + def registerTempActor(actorRef: InternalActorRef, path: ActorPath): Unit /** * Unregister a temporary actor from the “/temp” path (i.e. obtained from tempPath()); do NOT pass in any other path. */ - def unregisterTempActor(path: ActorPath) + def unregisterTempActor(path: ActorPath): Unit /** * Actor factory with create-only semantics: will create an actor as diff --git a/akka-actor/src/main/scala/akka/actor/Locker.scala b/akka-actor/src/main/scala/akka/actor/Locker.scala index efe98ab907..9e34f02332 100644 --- a/akka-actor/src/main/scala/akka/actor/Locker.scala +++ b/akka-actor/src/main/scala/akka/actor/Locker.scala @@ -9,7 +9,10 @@ import akka.util.duration._ import java.util.concurrent.ConcurrentHashMap import akka.event.DeathWatch -class Locker( +/** + * Internal implementation detail for disposing of orphaned actors. + */ +private[akka] class Locker( scheduler: Scheduler, period: Duration, val provider: ActorRefProvider, diff --git a/akka-actor/src/main/scala/akka/pattern/AskSupport.scala b/akka-actor/src/main/scala/akka/pattern/AskSupport.scala index 492bb46ed8..b9869a5126 100644 --- a/akka-actor/src/main/scala/akka/pattern/AskSupport.scala +++ b/akka-actor/src/main/scala/akka/pattern/AskSupport.scala @@ -19,9 +19,15 @@ class AskTimeoutException(message: String, cause: Throwable) extends TimeoutExce def this(message: String) = this(message, null: Throwable) } +/** + * This object contains implementation details of the “ask” pattern. + */ object AskSupport { - final class AskableActorRef(val actorRef: ActorRef) { + /** + * Implementation detail of the “ask” pattern enrichment of ActorRef + */ + private[akka] final class AskableActorRef(val actorRef: ActorRef) { /** * Sends a message asynchronously and returns a [[akka.dispatch.Future]] @@ -43,8 +49,8 @@ object AskSupport { * Recommended usage: * * {{{ - * val f = worker.ask(request)(timeout) * flow { + * val f = worker.ask(request)(timeout) * EnrichedRequest(request, f()) * } pipeTo nextActor * }}} @@ -73,8 +79,8 @@ object AskSupport { * Recommended usage: * * {{{ - * val f = worker ? request * flow { + * val f = worker ? request * EnrichedRequest(request, f()) * } pipeTo nextActor * }}} From 9d7ed5eba16152d15269bf4f626260432359c0d4 Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 23 Jan 2012 18:25:43 +0100 Subject: [PATCH 112/152] update copyright headers --- akka-actor/src/main/java/com/typesafe/config/Config.java | 2 +- .../src/main/java/com/typesafe/config/ConfigException.java | 2 +- akka-actor/src/main/java/com/typesafe/config/ConfigFactory.java | 2 +- .../src/main/java/com/typesafe/config/ConfigIncludeContext.java | 2 +- .../src/main/java/com/typesafe/config/ConfigIncluder.java | 2 +- akka-actor/src/main/java/com/typesafe/config/ConfigList.java | 2 +- .../src/main/java/com/typesafe/config/ConfigMergeable.java | 2 +- akka-actor/src/main/java/com/typesafe/config/ConfigObject.java | 2 +- akka-actor/src/main/java/com/typesafe/config/ConfigOrigin.java | 2 +- .../src/main/java/com/typesafe/config/ConfigParseOptions.java | 2 +- .../src/main/java/com/typesafe/config/ConfigParseable.java | 2 +- .../src/main/java/com/typesafe/config/ConfigResolveOptions.java | 2 +- akka-actor/src/main/java/com/typesafe/config/ConfigSyntax.java | 2 +- akka-actor/src/main/java/com/typesafe/config/ConfigValue.java | 2 +- .../src/main/java/com/typesafe/config/ConfigValueFactory.java | 2 +- .../src/main/java/com/typesafe/config/ConfigValueType.java | 2 +- .../java/com/typesafe/config/impl/AbstractConfigObject.java | 2 +- .../main/java/com/typesafe/config/impl/AbstractConfigValue.java | 2 +- .../src/main/java/com/typesafe/config/impl/ConfigBoolean.java | 2 +- .../main/java/com/typesafe/config/impl/ConfigDelayedMerge.java | 2 +- .../java/com/typesafe/config/impl/ConfigDelayedMergeObject.java | 2 +- .../src/main/java/com/typesafe/config/impl/ConfigDouble.java | 2 +- .../src/main/java/com/typesafe/config/impl/ConfigImpl.java | 2 +- .../src/main/java/com/typesafe/config/impl/ConfigImplUtil.java | 2 +- .../src/main/java/com/typesafe/config/impl/ConfigInt.java | 2 +- .../src/main/java/com/typesafe/config/impl/ConfigLong.java | 2 +- .../src/main/java/com/typesafe/config/impl/ConfigNull.java | 2 +- .../src/main/java/com/typesafe/config/impl/ConfigNumber.java | 2 +- .../src/main/java/com/typesafe/config/impl/ConfigString.java | 2 +- .../main/java/com/typesafe/config/impl/ConfigSubstitution.java | 2 +- .../main/java/com/typesafe/config/impl/DefaultTransformer.java | 2 +- .../src/main/java/com/typesafe/config/impl/FromMapMode.java | 2 +- .../src/main/java/com/typesafe/config/impl/Parseable.java | 2 +- akka-actor/src/main/java/com/typesafe/config/impl/Parser.java | 2 +- akka-actor/src/main/java/com/typesafe/config/impl/Path.java | 2 +- .../src/main/java/com/typesafe/config/impl/PathBuilder.java | 2 +- .../main/java/com/typesafe/config/impl/PropertiesParser.java | 2 +- .../src/main/java/com/typesafe/config/impl/ResolveStatus.java | 2 +- .../src/main/java/com/typesafe/config/impl/SimpleConfig.java | 2 +- .../main/java/com/typesafe/config/impl/SimpleConfigList.java | 2 +- .../main/java/com/typesafe/config/impl/SimpleConfigObject.java | 2 +- .../main/java/com/typesafe/config/impl/SimpleConfigOrigin.java | 2 +- .../java/com/typesafe/config/impl/SubstitutionResolver.java | 2 +- akka-actor/src/main/java/com/typesafe/config/impl/Token.java | 2 +- .../src/main/java/com/typesafe/config/impl/TokenType.java | 2 +- .../src/main/java/com/typesafe/config/impl/Tokenizer.java | 2 +- akka-actor/src/main/java/com/typesafe/config/impl/Tokens.java | 2 +- .../src/main/java/com/typesafe/config/impl/Unmergeable.java | 2 +- akka-actor/src/main/scala/akka/pattern/AskSupport.scala | 2 +- akka-actor/src/main/scala/akka/pattern/Patterns.scala | 2 +- akka-actor/src/main/scala/akka/pattern/PipeToSupport.scala | 2 +- akka-actor/src/main/scala/akka/pattern/package.scala | 2 +- akka-remote/src/multi-jvm/scala/akka/remote/ZKClient.scala | 2 +- akka-sbt-plugin/src/main/scala/AkkaKernelPlugin.scala | 2 +- 54 files changed, 54 insertions(+), 54 deletions(-) diff --git a/akka-actor/src/main/java/com/typesafe/config/Config.java b/akka-actor/src/main/java/com/typesafe/config/Config.java index d3496c73ef..e820d1b369 100644 --- a/akka-actor/src/main/java/com/typesafe/config/Config.java +++ b/akka-actor/src/main/java/com/typesafe/config/Config.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config; diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigException.java b/akka-actor/src/main/java/com/typesafe/config/ConfigException.java index b8dcb8ca00..3c31d811c3 100644 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigException.java +++ b/akka-actor/src/main/java/com/typesafe/config/ConfigException.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config; diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigFactory.java b/akka-actor/src/main/java/com/typesafe/config/ConfigFactory.java index df5e762a5c..bb3f0aa131 100644 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigFactory.java +++ b/akka-actor/src/main/java/com/typesafe/config/ConfigFactory.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config; diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigIncludeContext.java b/akka-actor/src/main/java/com/typesafe/config/ConfigIncludeContext.java index 2be0abff34..ac3644a5af 100644 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigIncludeContext.java +++ b/akka-actor/src/main/java/com/typesafe/config/ConfigIncludeContext.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config; diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigIncluder.java b/akka-actor/src/main/java/com/typesafe/config/ConfigIncluder.java index 1ac6f3383d..38e8e35a91 100644 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigIncluder.java +++ b/akka-actor/src/main/java/com/typesafe/config/ConfigIncluder.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config; diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigList.java b/akka-actor/src/main/java/com/typesafe/config/ConfigList.java index 2024efe744..5c694a508e 100644 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigList.java +++ b/akka-actor/src/main/java/com/typesafe/config/ConfigList.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config; diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigMergeable.java b/akka-actor/src/main/java/com/typesafe/config/ConfigMergeable.java index 1214db8c44..c760bf9d15 100644 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigMergeable.java +++ b/akka-actor/src/main/java/com/typesafe/config/ConfigMergeable.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config; diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigObject.java b/akka-actor/src/main/java/com/typesafe/config/ConfigObject.java index bb4d14da89..16b9837f6d 100644 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigObject.java +++ b/akka-actor/src/main/java/com/typesafe/config/ConfigObject.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config; diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigOrigin.java b/akka-actor/src/main/java/com/typesafe/config/ConfigOrigin.java index fbc1fe17c3..c34767fb2e 100644 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigOrigin.java +++ b/akka-actor/src/main/java/com/typesafe/config/ConfigOrigin.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config; diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigParseOptions.java b/akka-actor/src/main/java/com/typesafe/config/ConfigParseOptions.java index f3765a5479..2d057e812f 100644 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigParseOptions.java +++ b/akka-actor/src/main/java/com/typesafe/config/ConfigParseOptions.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config; diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigParseable.java b/akka-actor/src/main/java/com/typesafe/config/ConfigParseable.java index 1cc39614ca..8c19085513 100644 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigParseable.java +++ b/akka-actor/src/main/java/com/typesafe/config/ConfigParseable.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config; diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigResolveOptions.java b/akka-actor/src/main/java/com/typesafe/config/ConfigResolveOptions.java index 3adb589f1d..d82a6be71a 100644 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigResolveOptions.java +++ b/akka-actor/src/main/java/com/typesafe/config/ConfigResolveOptions.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config; diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigSyntax.java b/akka-actor/src/main/java/com/typesafe/config/ConfigSyntax.java index 7626a92e6d..54529fad0b 100644 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigSyntax.java +++ b/akka-actor/src/main/java/com/typesafe/config/ConfigSyntax.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config; diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigValue.java b/akka-actor/src/main/java/com/typesafe/config/ConfigValue.java index b636c6f4cd..1f389be08f 100644 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigValue.java +++ b/akka-actor/src/main/java/com/typesafe/config/ConfigValue.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config; diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigValueFactory.java b/akka-actor/src/main/java/com/typesafe/config/ConfigValueFactory.java index 14c2bff8f7..babace186e 100644 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigValueFactory.java +++ b/akka-actor/src/main/java/com/typesafe/config/ConfigValueFactory.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config; diff --git a/akka-actor/src/main/java/com/typesafe/config/ConfigValueType.java b/akka-actor/src/main/java/com/typesafe/config/ConfigValueType.java index 981cb6d189..a15774d3ce 100644 --- a/akka-actor/src/main/java/com/typesafe/config/ConfigValueType.java +++ b/akka-actor/src/main/java/com/typesafe/config/ConfigValueType.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigObject.java b/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigObject.java index a21ccd81f7..1eb31c6397 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigObject.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigObject.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigValue.java b/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigValue.java index e51f4c6067..46063f410e 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigValue.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigValue.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigBoolean.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigBoolean.java index c926c0c942..a1a7677c78 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigBoolean.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigBoolean.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMerge.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMerge.java index 4cca7834bd..218aaca306 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMerge.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMerge.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMergeObject.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMergeObject.java index fe970d59c8..a9ff7bd3bf 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMergeObject.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDelayedMergeObject.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDouble.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDouble.java index c26d3cd6a9..65cfb5d966 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDouble.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigDouble.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigImpl.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigImpl.java index 217f4385e9..73ddfdce5a 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigImpl.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigImpl.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigImplUtil.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigImplUtil.java index cbc0ecca09..4cbcff7aa3 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigImplUtil.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigImplUtil.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigInt.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigInt.java index 440b5ae8cf..d33bd21274 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigInt.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigInt.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigLong.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigLong.java index 6a72bc4cab..a2e2ab3228 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigLong.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigLong.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNull.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNull.java index fbdc21d7a5..91b8586674 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNull.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNull.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNumber.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNumber.java index 4a6bbd0b15..d2f4bb6ac4 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNumber.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigNumber.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigString.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigString.java index 9b41e7f7ab..f5293c8fc7 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigString.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigString.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigSubstitution.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigSubstitution.java index f4441b81a5..23776e0011 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigSubstitution.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigSubstitution.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/DefaultTransformer.java b/akka-actor/src/main/java/com/typesafe/config/impl/DefaultTransformer.java index 4391814acb..9a9bf5c6a9 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/DefaultTransformer.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/DefaultTransformer.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/FromMapMode.java b/akka-actor/src/main/java/com/typesafe/config/impl/FromMapMode.java index ce6c3e3f0a..764a45664c 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/FromMapMode.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/FromMapMode.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/Parseable.java b/akka-actor/src/main/java/com/typesafe/config/impl/Parseable.java index 62b8ee575a..4938603199 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/Parseable.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/Parseable.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/Parser.java b/akka-actor/src/main/java/com/typesafe/config/impl/Parser.java index 5df0314fe6..1ba8535207 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/Parser.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/Parser.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/Path.java b/akka-actor/src/main/java/com/typesafe/config/impl/Path.java index 193d930002..af6dfe8bf5 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/Path.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/Path.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/PathBuilder.java b/akka-actor/src/main/java/com/typesafe/config/impl/PathBuilder.java index f46e78201e..ede6c66387 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/PathBuilder.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/PathBuilder.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/PropertiesParser.java b/akka-actor/src/main/java/com/typesafe/config/impl/PropertiesParser.java index bd822e65e9..7c8c81fb07 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/PropertiesParser.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/PropertiesParser.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ResolveStatus.java b/akka-actor/src/main/java/com/typesafe/config/impl/ResolveStatus.java index 3f73eb5221..8deeaf520f 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ResolveStatus.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/ResolveStatus.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfig.java b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfig.java index 17979ba6cc..a87b12316a 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfig.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfig.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigList.java b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigList.java index 1921826352..fa91ea6627 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigList.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigList.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigObject.java b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigObject.java index 953f26491f..c961ae6b34 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigObject.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigObject.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigOrigin.java b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigOrigin.java index f0a0dbd353..4f8859f050 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigOrigin.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/SimpleConfigOrigin.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/SubstitutionResolver.java b/akka-actor/src/main/java/com/typesafe/config/impl/SubstitutionResolver.java index 65a2b30900..7bb3bf3a61 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/SubstitutionResolver.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/SubstitutionResolver.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/Token.java b/akka-actor/src/main/java/com/typesafe/config/impl/Token.java index afff3247d6..5f16d26e1d 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/Token.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/Token.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/TokenType.java b/akka-actor/src/main/java/com/typesafe/config/impl/TokenType.java index fc617d9ee2..7853c09445 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/TokenType.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/TokenType.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/Tokenizer.java b/akka-actor/src/main/java/com/typesafe/config/impl/Tokenizer.java index 280a028077..2fcee8e61a 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/Tokenizer.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/Tokenizer.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/Tokens.java b/akka-actor/src/main/java/com/typesafe/config/impl/Tokens.java index d726d83d53..83bec62af3 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/Tokens.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/Tokens.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/Unmergeable.java b/akka-actor/src/main/java/com/typesafe/config/impl/Unmergeable.java index e0d114e78d..0028f2e023 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/Unmergeable.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/Unmergeable.java @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe Inc. + * Copyright (C) 2011-2012 Typesafe Inc. */ package com.typesafe.config.impl; diff --git a/akka-actor/src/main/scala/akka/pattern/AskSupport.scala b/akka-actor/src/main/scala/akka/pattern/AskSupport.scala index b9869a5126..3e637fc81d 100644 --- a/akka-actor/src/main/scala/akka/pattern/AskSupport.scala +++ b/akka-actor/src/main/scala/akka/pattern/AskSupport.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.pattern diff --git a/akka-actor/src/main/scala/akka/pattern/Patterns.scala b/akka-actor/src/main/scala/akka/pattern/Patterns.scala index 8c52f096f1..c3510d9b68 100644 --- a/akka-actor/src/main/scala/akka/pattern/Patterns.scala +++ b/akka-actor/src/main/scala/akka/pattern/Patterns.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.pattern diff --git a/akka-actor/src/main/scala/akka/pattern/PipeToSupport.scala b/akka-actor/src/main/scala/akka/pattern/PipeToSupport.scala index 6827946902..26f3b68e38 100644 --- a/akka-actor/src/main/scala/akka/pattern/PipeToSupport.scala +++ b/akka-actor/src/main/scala/akka/pattern/PipeToSupport.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.pattern diff --git a/akka-actor/src/main/scala/akka/pattern/package.scala b/akka-actor/src/main/scala/akka/pattern/package.scala index 2689b72548..ac8fcf2df2 100644 --- a/akka-actor/src/main/scala/akka/pattern/package.scala +++ b/akka-actor/src/main/scala/akka/pattern/package.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/ZKClient.scala b/akka-remote/src/multi-jvm/scala/akka/remote/ZKClient.scala index 96bccb13fa..156c955566 100755 --- a/akka-remote/src/multi-jvm/scala/akka/remote/ZKClient.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/ZKClient.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe + * Copyright (C) 2011-2012 Typesafe */ package akka.remote diff --git a/akka-sbt-plugin/src/main/scala/AkkaKernelPlugin.scala b/akka-sbt-plugin/src/main/scala/AkkaKernelPlugin.scala index ad6d2f13ea..5031723f44 100644 --- a/akka-sbt-plugin/src/main/scala/AkkaKernelPlugin.scala +++ b/akka-sbt-plugin/src/main/scala/AkkaKernelPlugin.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2011 Typesafe + * Copyright (C) 2011-2012 Typesafe */ package akka.sbt From 6d348b6e0e8b8fe392081a0fa737f850aef175b6 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Mon, 23 Jan 2012 10:40:05 -0700 Subject: [PATCH 113/152] IO: Support random port --- .../src/test/scala/akka/actor/IOActor.scala | 24 ++++++++++++------- akka-actor/src/main/scala/akka/actor/IO.scala | 17 +++++++------ 2 files changed, 25 insertions(+), 16 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala index e01729d632..fbea3cc89e 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala @@ -9,6 +9,7 @@ import akka.util.duration._ import scala.util.continuations._ import akka.testkit._ import akka.dispatch.{ Await, Future, Promise, ExecutionContext, MessageDispatcher } +import java.net.{ SocketAddress, InetSocketAddress } object IOActorSpec { @@ -58,7 +59,7 @@ object IOActorSpec { case IO.Read(`socket`, bytes) ⇒ state(IO Chunk bytes) - case IO.Connected(`socket`) ⇒ + case IO.Connected(`socket`, _) ⇒ case IO.Closed(`socket`, cause) ⇒ state(IO EOF cause) @@ -89,7 +90,7 @@ object IOActorSpec { } // Basic Redis-style protocol - class KVStore(host: String, port: Int) extends Actor { + class KVStore(addressPromise: Promise[SocketAddress]) extends Actor { import context.system @@ -97,12 +98,15 @@ object IOActorSpec { var kvs: Map[String, String] = Map.empty - val server = IOManager(context.system) listen (host, port) + val server = IOManager(context.system) listen (new InetSocketAddress("localhost", 0)) val EOL = ByteString("\r\n") def receive = { + case IO.Listening(`server`, address) ⇒ + addressPromise success address + case IO.NewClient(`server`) ⇒ val socket = server.accept() state(socket) flatMap { _ ⇒ @@ -157,9 +161,9 @@ object IOActorSpec { } } - class KVClient(host: String, port: Int) extends Actor { + class KVClient(address: SocketAddress) extends Actor { - val socket = IOManager(context.system) connect (host, port) + val socket = IOManager(context.system) connect (address) val state = IO.IterateeRef.sync() @@ -178,7 +182,7 @@ object IOActorSpec { case IO.Read(`socket`, bytes) ⇒ state(IO Chunk bytes) - case IO.Connected(`socket`) ⇒ + case IO.Connected(`socket`, _) ⇒ case IO.Closed(`socket`, cause) ⇒ state(IO EOF cause) @@ -296,9 +300,11 @@ class IOActorSpec extends AkkaSpec with DefaultTimeout { "run key-value store" in { filterException[java.net.ConnectException] { - val server = system.actorOf(Props(new KVStore("localhost", 8067))) - val client1 = system.actorOf(Props(new KVClient("localhost", 8067))) - val client2 = system.actorOf(Props(new KVClient("localhost", 8067))) + val addressPromise = Promise[SocketAddress]() + val server = system.actorOf(Props(new KVStore(addressPromise))) + val address = Await.result(addressPromise, TestLatch.DefaultTimeout) + val client1 = system.actorOf(Props(new KVClient(address))) + val client2 = system.actorOf(Props(new KVClient(address))) val f1 = retry() { client1 ? KVSet("hello", "World") } val f2 = retry() { client1 ? KVSet("test", "No one will read me") } val f3 = f1 flatMap { _ ⇒ retry() { client1 ? KVGet("hello") } } diff --git a/akka-actor/src/main/scala/akka/actor/IO.scala b/akka-actor/src/main/scala/akka/actor/IO.scala index 176501d5b8..52addf9bfd 100644 --- a/akka-actor/src/main/scala/akka/actor/IO.scala +++ b/akka-actor/src/main/scala/akka/actor/IO.scala @@ -5,7 +5,7 @@ package akka.actor import akka.dispatch.{ Future, ExecutionContext } import akka.util.ByteString -import java.net.InetSocketAddress +import java.net.{ SocketAddress, InetSocketAddress } import java.io.IOException import java.nio.ByteBuffer import java.nio.channels.{ @@ -132,7 +132,9 @@ object IO { * * Normally sent using IOManager.listen() */ - case class Listen(server: ServerHandle, address: InetSocketAddress) extends IOMessage + case class Listen(server: ServerHandle, address: SocketAddress) extends IOMessage + + case class Listening(server: ServerHandle, address: SocketAddress) extends IOMessage /** * Message from an [[akka.actor.IOManager]] that a new connection has been @@ -153,7 +155,7 @@ object IO { * * Normally sent using IOManager.connect() */ - case class Connect(socket: SocketHandle, address: InetSocketAddress) extends IOMessage + case class Connect(socket: SocketHandle, address: SocketAddress) extends IOMessage /** * Message from an [[akka.actor.IOManager]] that the SocketChannel has @@ -162,7 +164,7 @@ object IO { * No action is required by the receiving [[akka.actor.Actor]], although * the message still needs to be in it's receive method. */ - case class Connected(socket: SocketHandle) extends IOMessage + case class Connected(socket: SocketHandle, address: SocketAddress) extends IOMessage /** * Message to an [[akka.actor.IOManager]] to close the Channel. @@ -687,7 +689,7 @@ final class IOManager private (system: ActorSystem) extends Extension { * @param owner the ActorRef that will receive messages from the IOManagerActor * @return a [[akka.actor.IO.ServerHandle]] to uniquely identify the created socket */ - def listen(address: InetSocketAddress)(implicit owner: ActorRef): IO.ServerHandle = { + def listen(address: SocketAddress)(implicit owner: ActorRef): IO.ServerHandle = { val server = IO.ServerHandle(owner, actor) actor ! IO.Listen(server, address) server @@ -715,7 +717,7 @@ final class IOManager private (system: ActorSystem) extends Extension { * @param owner the ActorRef that will receive messages from the IOManagerActor * @return a [[akka.actor.IO.SocketHandle]] to uniquely identify the created socket */ - def connect(address: InetSocketAddress)(implicit owner: ActorRef): IO.SocketHandle = { + def connect(address: SocketAddress)(implicit owner: ActorRef): IO.SocketHandle = { val socket = IO.SocketHandle(owner, actor) actor ! IO.Connect(socket, address) socket @@ -834,6 +836,7 @@ final class IOManagerActor extends Actor { channel.socket bind (address, 1000) // TODO: make backlog configurable channels update (server, channel) channel register (selector, OP_ACCEPT, server) + server.owner ! IO.Listening(server, channel.getLocalAddress()) run() case IO.Connect(socket, address) ⇒ @@ -948,7 +951,7 @@ final class IOManagerActor extends Actor { private def connect(socket: IO.SocketHandle, channel: SocketChannel) { if (channel.finishConnect) { removeOps(socket, OP_CONNECT) - socket.owner ! IO.Connected(socket) + socket.owner ! IO.Connected(socket, channel.getLocalAddress()) } else { cleanup(socket, None) // TODO: Add a cause } From 4c65c58bce27b4caf6707f5156f5fc3496e935c9 Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 23 Jan 2012 18:52:40 +0100 Subject: [PATCH 114/152] remove implementation from TypedActor lifecycle traits to require their implementation --- akka-actor/src/main/scala/akka/actor/TypedActor.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index 83a7c0aa62..28c4363788 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -288,7 +288,7 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi * User overridable definition the strategy to use for supervising * child actors. */ - def supervisorStrategy(): SupervisorStrategy = SupervisorStrategy.defaultStrategy + def supervisorStrategy(): SupervisorStrategy } /** @@ -300,7 +300,7 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi *

* Is called when an Actor is started by invoking 'actor'. */ - def preStart(): Unit = () + def preStart(): Unit } /** @@ -312,7 +312,7 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi *

* Is called when 'actor.stop()' is invoked. */ - def postStop(): Unit = () + def postStop(): Unit } /** @@ -326,7 +326,7 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi * up of resources before Actor is terminated. * By default it calls postStop() */ - def preRestart(reason: Throwable, message: Option[Any]): Unit = () + def preRestart(reason: Throwable, message: Option[Any]): Unit } trait PostRestart { @@ -336,7 +336,7 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi * Is called right AFTER restart on the newly created Actor to allow reinitialization after an Actor crash. * By default it calls preStart() */ - def postRestart(reason: Throwable): Unit = () + def postRestart(reason: Throwable): Unit } private[akka] class TypedActorInvocationHandler(val extension: TypedActorExtension, val actorVar: AtomVar[ActorRef], val timeout: Timeout) extends InvocationHandler { From 4d3369ec5a950e27afe81d26b416801710045b03 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Mon, 23 Jan 2012 11:12:15 -0700 Subject: [PATCH 115/152] IO: Don't use JDK7 api --- akka-actor/src/main/scala/akka/actor/IO.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/IO.scala b/akka-actor/src/main/scala/akka/actor/IO.scala index 52addf9bfd..b7bfe9f0b4 100644 --- a/akka-actor/src/main/scala/akka/actor/IO.scala +++ b/akka-actor/src/main/scala/akka/actor/IO.scala @@ -836,7 +836,7 @@ final class IOManagerActor extends Actor { channel.socket bind (address, 1000) // TODO: make backlog configurable channels update (server, channel) channel register (selector, OP_ACCEPT, server) - server.owner ! IO.Listening(server, channel.getLocalAddress()) + server.owner ! IO.Listening(server, channel.socket.getLocalSocketAddress()) run() case IO.Connect(socket, address) ⇒ @@ -951,7 +951,7 @@ final class IOManagerActor extends Actor { private def connect(socket: IO.SocketHandle, channel: SocketChannel) { if (channel.finishConnect) { removeOps(socket, OP_CONNECT) - socket.owner ! IO.Connected(socket, channel.getLocalAddress()) + socket.owner ! IO.Connected(socket, channel.socket.getRemoteSocketAddress()) } else { cleanup(socket, None) // TODO: Add a cause } From 2615d0c731138b6c6392ad1eaa0d2cf54784ab45 Mon Sep 17 00:00:00 2001 From: Peter Vlugter Date: Tue, 24 Jan 2012 12:28:05 +1300 Subject: [PATCH 116/152] Update zeromq module - rewrite ZeroMQ java api object in scala to fix scaladoc errors - move package object to correct dir - update copyright headers --- .../src/main/java/akka/zeromq/ZeroMQ.java | 260 ------------------ .../src/main/scala/akka/zeromq/Response.scala | 2 +- .../main/scala/akka/zeromq/SocketOption.scala | 2 +- .../akka/zeromq/ZMQMessageDeserializer.scala | 2 +- .../src/main/scala/akka/zeromq/ZeroMQ.scala | 223 +++++++++++++++ .../scala/akka/zeromq/ZeroMQExtension.scala | 2 +- .../scala/{ => akka/zeromq}/package.scala | 3 + .../java/akka/zeromq/ZeroMQFromJavaTests.java | 32 +++ .../akka/zeromq/ZeroMQFromJavaSpec.scala | 8 + 9 files changed, 270 insertions(+), 264 deletions(-) delete mode 100644 akka-zeromq/src/main/java/akka/zeromq/ZeroMQ.java create mode 100644 akka-zeromq/src/main/scala/akka/zeromq/ZeroMQ.scala rename akka-zeromq/src/main/scala/{ => akka/zeromq}/package.scala (89%) create mode 100644 akka-zeromq/src/test/java/akka/zeromq/ZeroMQFromJavaTests.java create mode 100644 akka-zeromq/src/test/scala/akka/zeromq/ZeroMQFromJavaSpec.scala diff --git a/akka-zeromq/src/main/java/akka/zeromq/ZeroMQ.java b/akka-zeromq/src/main/java/akka/zeromq/ZeroMQ.java deleted file mode 100644 index e539af18af..0000000000 --- a/akka-zeromq/src/main/java/akka/zeromq/ZeroMQ.java +++ /dev/null @@ -1,260 +0,0 @@ -package akka.zeromq; - -/** - * Java API for akka.zeromq - */ -public class ZeroMQ { - - /** - * The message that is sent when an ZeroMQ socket connects. - *

- *

-     * if (message == connecting()) {
-     *   // Socket connected
-     * }
-     * 
- * - * @return the single instance of Connecting - */ - public final static Connecting$ connecting() { - return Connecting$.MODULE$; - } - - /** - * The message that is sent when an ZeroMQ socket disconnects. - *

- *

-     * if (message == closed()) {
-     *   // Socket disconnected
-     * }
-     * 
- * - * @return the single instance of Closed - */ - public final static Closed$ closed() { - return Closed$.MODULE$; - } - - /** - * The message to ask a ZeroMQ socket for its affinity configuration. - *

- *

-     * socket.ask(affinity())
-     * 
- * - * @return the single instance of Affinity - */ - public final static Affinity$ affinity() { - return Affinity$.MODULE$; - } - - /** - * The message to ask a ZeroMQ socket for its backlog configuration. - *

- *

-     * socket.ask(backlog())
-     * 
- * - * @return the single instance of Backlog - */ - public final static Backlog$ backlog() { - return Backlog$.MODULE$; - } - - /** - * The message to ask a ZeroMQ socket for its file descriptor configuration. - *

- *

-     * socket.ask(fileDescriptor())
-     * 
- * - * @return the single instance of FileDescriptor - */ - public final static FileDescriptor$ fileDescriptor() { - return FileDescriptor$.MODULE$; - } - - /** - * The message to ask a ZeroMQ socket for its identity configuration. - *

- *

-     * socket.ask(identity())
-     * 
- * - * @return the single instance of Identity - */ - public final static Identity$ identity() { - return Identity$.MODULE$; - } - - /** - * The message to ask a ZeroMQ socket for its linger configuration. - *

- *

-     * socket.ask(linger())
-     * 
- * - * @return the single instance of Linger - */ - public final static Linger$ linger() { - return Linger$.MODULE$; - } - - /** - * The message to ask a ZeroMQ socket for its max message size configuration. - *

- *

-     * socket.ask(maxMessageSize())
-     * 
- * - * @return the single instance of MaxMsgSize - */ - public final static MaxMsgSize$ maxMessageSize() { - return MaxMsgSize$.MODULE$; - } - - /** - * The message to ask a ZeroMQ socket for its multicast hops configuration. - *

- *

-     * socket.ask(multicastHops())
-     * 
- * - * @return the single instance of MulticastHops - */ - public final static MulticastHops$ multicastHops() { - return MulticastHops$.MODULE$; - } - - /** - * The message to ask a ZeroMQ socket for its multicast loop configuration. - *

- *

-     * socket.ask(multicastLoop())
-     * 
- * - * @return the single instance of MulticastLoop - */ - public final static MulticastLoop$ multicastLoop() { - return MulticastLoop$.MODULE$; - } - - /** - * The message to ask a ZeroMQ socket for its rate configuration. - *

- *

-     * socket.ask(rate())
-     * 
- * - * @return the single instance of Rate - */ - public final static Rate$ rate() { - return Rate$.MODULE$; - } - - /** - * The message to ask a ZeroMQ socket for its receive bufferSize configuration. - *

- *

-     * socket.ask(receiveBufferSize())
-     * 
- * - * @return the single instance of ReceiveBufferSize - */ - public final static ReceiveBufferSize$ receiveBufferSize() { - return ReceiveBufferSize$.MODULE$; - } - - /** - * The message to ask a ZeroMQ socket for its receive high watermark configuration. - *

- *

-     * socket.ask(receiveHighWatermark())
-     * 
- * - * @return the single instance of ReceiveHighWatermark - */ - public final static ReceiveHighWatermark$ receiveHighWatermark() { - return ReceiveHighWatermark$.MODULE$; - } - - /** - * The message to ask a ZeroMQ socket for its reconnect interval configuration. - *

- *

-     * socket.ask(reconnectIVL())
-     * 
- * - * @return the single instance of ReconnectIVL - */ - public final static ReconnectIVL$ reconnectIVL() { - return ReconnectIVL$.MODULE$; - } - - /** - * The message to ask a ZeroMQ socket for its max reconnect interval configuration. - *

- *

-     * socket.ask(reconnectIVLMax())
-     * 
- * - * @return the single instance of ReconnectIVLMax - */ - public final static ReconnectIVLMax$ reconnectIVLMax() { - return ReconnectIVLMax$.MODULE$; - } - - /** - * The message to ask a ZeroMQ socket for its recovery interval configuration. - *

- *

-     * socket.ask(recoveryInterval())
-     * 
- * - * @return the single instance of RecoveryInterval - */ - public final static RecoveryInterval$ recoveryInterval() { - return RecoveryInterval$.MODULE$; - } - - /** - * The message to ask a ZeroMQ socket for its send buffer size configuration. - *

- *

-     * socket.ask(sendBufferSize())
-     * 
- * - * @return the single instance of SendBufferSize - */ - public final static SendBufferSize$ sendBufferSize() { - return SendBufferSize$.MODULE$; - } - - /** - * The message to ask a ZeroMQ socket for its send high watermark configuration. - *

- *

-     * socket.ask(sendHighWatermark())
-     * 
- * - * @return the single instance of SendHighWatermark - */ - public final static SendHighWatermark$ sendHighWatermark() { - return SendHighWatermark$.MODULE$; - } - - /** - * The message to ask a ZeroMQ socket for its swap configuration. - *

- *

-     * socket.ask(swap())
-     * 
- * - * @return the single instance of Swap - */ - public final static Swap$ swap() { - return Swap$.MODULE$; - } - - -} diff --git a/akka-zeromq/src/main/scala/akka/zeromq/Response.scala b/akka-zeromq/src/main/scala/akka/zeromq/Response.scala index f916ba0a4f..e9ef273397 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/Response.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/Response.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.zeromq diff --git a/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala b/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala index 488b99fa22..9cded84da1 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/SocketOption.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.zeromq diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ZMQMessageDeserializer.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZMQMessageDeserializer.scala index 3acef255a7..470c98617a 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ZMQMessageDeserializer.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZMQMessageDeserializer.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.zeromq diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQ.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQ.scala new file mode 100644 index 0000000000..df581d6a37 --- /dev/null +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQ.scala @@ -0,0 +1,223 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.zeromq; + +/** + * Java API for akka.zeromq + */ +object ZeroMQ { + + /** + * The message that is sent when an ZeroMQ socket connects. + *

+ *

+   * if (message == connecting()) {
+   *   // Socket connected
+   * }
+   * 
+ * + * @return the single instance of Connecting + */ + def connecting() = Connecting + + /** + * The message that is sent when an ZeroMQ socket disconnects. + *

+ *

+   * if (message == closed()) {
+   *   // Socket disconnected
+   * }
+   * 
+ * + * @return the single instance of Closed + */ + def closed() = Closed + + /** + * The message to ask a ZeroMQ socket for its affinity configuration. + *

+ *

+   * socket.ask(affinity())
+   * 
+ * + * @return the single instance of Affinity + */ + def affinity() = Affinity + + /** + * The message to ask a ZeroMQ socket for its backlog configuration. + *

+ *

+   * socket.ask(backlog())
+   * 
+ * + * @return the single instance of Backlog + */ + def backlog() = Backlog + + /** + * The message to ask a ZeroMQ socket for its file descriptor configuration. + *

+ *

+   * socket.ask(fileDescriptor())
+   * 
+ * + * @return the single instance of FileDescriptor + */ + def fileDescriptor() = FileDescriptor + + /** + * The message to ask a ZeroMQ socket for its identity configuration. + *

+ *

+   * socket.ask(identity())
+   * 
+ * + * @return the single instance of Identity + */ + def identity() = Identity + + /** + * The message to ask a ZeroMQ socket for its linger configuration. + *

+ *

+   * socket.ask(linger())
+   * 
+ * + * @return the single instance of Linger + */ + def linger() = Linger + + /** + * The message to ask a ZeroMQ socket for its max message size configuration. + *

+ *

+   * socket.ask(maxMessageSize())
+   * 
+ * + * @return the single instance of MaxMsgSize + */ + def maxMessageSize() = MaxMsgSize + + /** + * The message to ask a ZeroMQ socket for its multicast hops configuration. + *

+ *

+   * socket.ask(multicastHops())
+   * 
+ * + * @return the single instance of MulticastHops + */ + def multicastHops() = MulticastHops + + /** + * The message to ask a ZeroMQ socket for its multicast loop configuration. + *

+ *

+   * socket.ask(multicastLoop())
+   * 
+ * + * @return the single instance of MulticastLoop + */ + def multicastLoop() = MulticastLoop + + /** + * The message to ask a ZeroMQ socket for its rate configuration. + *

+ *

+   * socket.ask(rate())
+   * 
+ * + * @return the single instance of Rate + */ + def rate() = Rate + + /** + * The message to ask a ZeroMQ socket for its receive bufferSize configuration. + *

+ *

+   * socket.ask(receiveBufferSize())
+   * 
+ * + * @return the single instance of ReceiveBufferSize + */ + def receiveBufferSize() = ReceiveBufferSize + + /** + * The message to ask a ZeroMQ socket for its receive high watermark configuration. + *

+ *

+   * socket.ask(receiveHighWatermark())
+   * 
+ * + * @return the single instance of ReceiveHighWatermark + */ + def receiveHighWatermark() = ReceiveHighWatermark + + /** + * The message to ask a ZeroMQ socket for its reconnect interval configuration. + *

+ *

+   * socket.ask(reconnectIVL())
+   * 
+ * + * @return the single instance of ReconnectIVL + */ + def reconnectIVL() = ReconnectIVL + + /** + * The message to ask a ZeroMQ socket for its max reconnect interval configuration. + *

+ *

+   * socket.ask(reconnectIVLMax())
+   * 
+ * + * @return the single instance of ReconnectIVLMax + */ + def reconnectIVLMax() = ReconnectIVLMax + + /** + * The message to ask a ZeroMQ socket for its recovery interval configuration. + *

+ *

+   * socket.ask(recoveryInterval())
+   * 
+ * + * @return the single instance of RecoveryInterval + */ + def recoveryInterval() = RecoveryInterval + + /** + * The message to ask a ZeroMQ socket for its send buffer size configuration. + *

+ *

+   * socket.ask(sendBufferSize())
+   * 
+ * + * @return the single instance of SendBufferSize + */ + def sendBufferSize() = SendBufferSize + + /** + * The message to ask a ZeroMQ socket for its send high watermark configuration. + *

+ *

+   * socket.ask(sendHighWatermark())
+   * 
+ * + * @return the single instance of SendHighWatermark + */ + def sendHighWatermark() = SendHighWatermark + + /** + * The message to ask a ZeroMQ socket for its swap configuration. + *

+ *

+   * socket.ask(swap())
+   * 
+ * + * @return the single instance of Swap + */ + def swap() = Swap +} \ No newline at end of file diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala index 0a7b53bc8d..f0dee326b2 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala @@ -1,5 +1,5 @@ /** - * Copyright (C) 2009-2011 Typesafe Inc. + * Copyright (C) 2009-2012 Typesafe Inc. */ package akka.zeromq diff --git a/akka-zeromq/src/main/scala/package.scala b/akka-zeromq/src/main/scala/akka/zeromq/package.scala similarity index 89% rename from akka-zeromq/src/main/scala/package.scala rename to akka-zeromq/src/main/scala/akka/zeromq/package.scala index 11a62b3540..812c3eef50 100644 --- a/akka-zeromq/src/main/scala/package.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/package.scala @@ -1,3 +1,6 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ package akka import actor.ActorSystem diff --git a/akka-zeromq/src/test/java/akka/zeromq/ZeroMQFromJavaTests.java b/akka-zeromq/src/test/java/akka/zeromq/ZeroMQFromJavaTests.java new file mode 100644 index 0000000000..3fd5e96aa8 --- /dev/null +++ b/akka-zeromq/src/test/java/akka/zeromq/ZeroMQFromJavaTests.java @@ -0,0 +1,32 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.zeromq; + +import static org.junit.Assert.*; +import org.junit.Test; + +public class ZeroMQFromJavaTests { + @Test + public void checkObjectHelperMethods() { + assertTrue(ZeroMQ.connecting() == Connecting$.MODULE$); + assertTrue(ZeroMQ.closed() == Closed$.MODULE$); + assertTrue(ZeroMQ.affinity() == Affinity$.MODULE$); + assertTrue(ZeroMQ.backlog() == Backlog$.MODULE$); + assertTrue(ZeroMQ.fileDescriptor() == FileDescriptor$.MODULE$); + assertTrue(ZeroMQ.identity() == Identity$.MODULE$); + assertTrue(ZeroMQ.linger() == Linger$.MODULE$); + assertTrue(ZeroMQ.maxMessageSize() == MaxMsgSize$.MODULE$); + assertTrue(ZeroMQ.multicastHops() == MulticastHops$.MODULE$); + assertTrue(ZeroMQ.multicastLoop() == MulticastLoop$.MODULE$); + assertTrue(ZeroMQ.rate() == Rate$.MODULE$); + assertTrue(ZeroMQ.receiveBufferSize() == ReceiveBufferSize$.MODULE$); + assertTrue(ZeroMQ.receiveHighWatermark() == ReceiveHighWatermark$.MODULE$); + assertTrue(ZeroMQ.reconnectIVL() == ReconnectIVL$.MODULE$); + assertTrue(ZeroMQ.reconnectIVLMax() == ReconnectIVLMax$.MODULE$); + assertTrue(ZeroMQ.recoveryInterval() == RecoveryInterval$.MODULE$); + assertTrue(ZeroMQ.sendBufferSize() == SendBufferSize$.MODULE$); + assertTrue(ZeroMQ.sendHighWatermark() == SendHighWatermark$.MODULE$); + assertTrue(ZeroMQ.swap() == Swap$.MODULE$); + } +} diff --git a/akka-zeromq/src/test/scala/akka/zeromq/ZeroMQFromJavaSpec.scala b/akka-zeromq/src/test/scala/akka/zeromq/ZeroMQFromJavaSpec.scala new file mode 100644 index 0000000000..63a958060e --- /dev/null +++ b/akka-zeromq/src/test/scala/akka/zeromq/ZeroMQFromJavaSpec.scala @@ -0,0 +1,8 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.zeromq + +import org.scalatest.junit.JUnitWrapperSuite + +class ZeroMQFromJavaSpec extends JUnitWrapperSuite("akka.zeromq.ZeroMQFromJavaTests", Thread.currentThread.getContextClassLoader) From ae79194118d94e4cfba0f9e30aba4cadd9be9ada Mon Sep 17 00:00:00 2001 From: Peter Vlugter Date: Tue, 24 Jan 2012 12:35:32 +1300 Subject: [PATCH 117/152] Fix scaladoc @returns warnings (use @return) --- akka-actor/src/main/scala/akka/actor/TypedActor.scala | 10 +++++----- akka-actor/src/main/scala/akka/dispatch/Future.scala | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index 28c4363788..3293add151 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -376,7 +376,7 @@ object TypedProps { val defaultLoader: Option[ClassLoader] = None /** - * @returns a sequence of interfaces that the speicified class implements, + * @return a sequence of interfaces that the speicified class implements, * or a sequence containing only itself, if itself is an interface. */ def extractInterfaces(clazz: Class[_]): Seq[Class[_]] = @@ -474,7 +474,7 @@ case class TypedProps[T <: AnyRef] protected[TypedProps] ( def withDispatcher(d: String) = copy(dispatcher = d) /** - * @returns a new Props that will use the specified ClassLoader to create its proxy class in + * @return a new Props that will use the specified ClassLoader to create its proxy class in * If loader is null, it will use the bootstrap classloader. * * Java API @@ -482,7 +482,7 @@ case class TypedProps[T <: AnyRef] protected[TypedProps] ( def withLoader(loader: ClassLoader): TypedProps[T] = withLoader(Option(loader)) /** - * @returns a new Props that will use the specified ClassLoader to create its proxy class in + * @return a new Props that will use the specified ClassLoader to create its proxy class in * If loader is null, it will use the bootstrap classloader. * * Scala API @@ -490,7 +490,7 @@ case class TypedProps[T <: AnyRef] protected[TypedProps] ( def withLoader(loader: Option[ClassLoader]): TypedProps[T] = this.copy(loader = loader) /** - * @returns a new Props that will use the specified Timeout for its non-void-returning methods, + * @return a new Props that will use the specified Timeout for its non-void-returning methods, * if null is specified, it will use the default ActorTimeout as specified in the configuration. * * Java API @@ -498,7 +498,7 @@ case class TypedProps[T <: AnyRef] protected[TypedProps] ( def withTimeout(timeout: Timeout): TypedProps[T] = this.copy(timeout = Option(timeout)) /** - * @returns a new Props that will use the specified Timeout for its non-void-returning methods, + * @return a new Props that will use the specified Timeout for its non-void-returning methods, * if None is specified, it will use the default ActorTimeout as specified in the configuration. * * Scala API diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 63dcbf6a35..09ce22d6b8 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -354,7 +354,7 @@ sealed trait Future[+T] extends japi.Future[T] with Await.Awaitable[T] { } /** - * @returns a new Future that will contain a tuple containing the successful result of this and that Future. + * @return a new Future that will contain a tuple containing the successful result of this and that Future. * If this or that fail, they will race to complete the returned Future with their failure. * The returned Future will not be completed if neither this nor that are completed. */ From 4bbae37ee703ffd46384d9983b21e3cff205a449 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Mon, 23 Jan 2012 23:44:28 -0700 Subject: [PATCH 118/152] Use random port in remaining IO tests --- .../src/test/scala/akka/actor/IOActor.scala | 31 ++++++++++--------- akka-actor/src/main/scala/akka/actor/IO.scala | 12 ++++--- 2 files changed, 25 insertions(+), 18 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala index 7ef9f1c229..13ed9d8c7e 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala @@ -9,19 +9,22 @@ import akka.util.duration._ import scala.util.continuations._ import akka.testkit._ import akka.dispatch.{ Await, Future, Promise, ExecutionContext, MessageDispatcher } -import java.net.{ SocketAddress, InetSocketAddress } +import java.net.{ SocketAddress } import akka.pattern.ask object IOActorSpec { - class SimpleEchoServer(host: String, port: Int) extends Actor { + class SimpleEchoServer(addressPromise: Promise[SocketAddress]) extends Actor { - val server = IOManager(context.system) listen (host, port) + val server = IOManager(context.system) listen ("localhost", 0) val state = IO.IterateeRef.Map.sync[IO.Handle]() def receive = { + case IO.Listening(`server`, address) ⇒ + addressPromise success address + case IO.NewClient(`server`) ⇒ val socket = server.accept() state(socket) flatMap (_ ⇒ IO repeat (IO.takeAny map socket.write)) @@ -40,9 +43,9 @@ object IOActorSpec { } } - class SimpleEchoClient(host: String, port: Int) extends Actor { + class SimpleEchoClient(address: SocketAddress) extends Actor { - val socket = IOManager(context.system) connect (host, port) + val socket = IOManager(context.system) connect (address) val state = IO.IterateeRef.sync() @@ -60,8 +63,6 @@ object IOActorSpec { case IO.Read(`socket`, bytes) ⇒ state(IO Chunk bytes) - case IO.Connected(`socket`, _) ⇒ - case IO.Closed(`socket`, cause) ⇒ state(IO EOF cause) throw (cause getOrElse new RuntimeException("Socket closed")) @@ -99,7 +100,7 @@ object IOActorSpec { var kvs: Map[String, String] = Map.empty - val server = IOManager(context.system) listen (new InetSocketAddress("localhost", 0)) + val server = IOManager(context.system) listen ("localhost", 0) val EOL = ByteString("\r\n") @@ -183,8 +184,6 @@ object IOActorSpec { case IO.Read(`socket`, bytes) ⇒ state(IO Chunk bytes) - case IO.Connected(`socket`, _) ⇒ - case IO.Closed(`socket`, cause) ⇒ state(IO EOF cause) throw (cause getOrElse new RuntimeException("Socket closed")) @@ -274,8 +273,10 @@ class IOActorSpec extends AkkaSpec with DefaultTimeout { "an IO Actor" must { "run echo server" in { filterException[java.net.ConnectException] { - val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8064))) - val client = system.actorOf(Props(new SimpleEchoClient("localhost", 8064))) + val addressPromise = Promise[SocketAddress]() + val server = system.actorOf(Props(new SimpleEchoServer(addressPromise))) + val address = Await.result(addressPromise, TestLatch.DefaultTimeout) + val client = system.actorOf(Props(new SimpleEchoClient(address))) val f1 = retry() { client ? ByteString("Hello World!1") } val f2 = retry() { client ? ByteString("Hello World!2") } val f3 = retry() { client ? ByteString("Hello World!3") } @@ -289,8 +290,10 @@ class IOActorSpec extends AkkaSpec with DefaultTimeout { "run echo server under high load" in { filterException[java.net.ConnectException] { - val server = system.actorOf(Props(new SimpleEchoServer("localhost", 8065))) - val client = system.actorOf(Props(new SimpleEchoClient("localhost", 8065))) + val addressPromise = Promise[SocketAddress]() + val server = system.actorOf(Props(new SimpleEchoServer(addressPromise))) + val address = Await.result(addressPromise, TestLatch.DefaultTimeout) + val client = system.actorOf(Props(new SimpleEchoClient(address))) val list = List.range(0, 100) val f = Future.traverse(list)(i ⇒ retry() { client ? ByteString(i.toString) }) assert(Await.result(f, TestLatch.DefaultTimeout).size === 100) diff --git a/akka-actor/src/main/scala/akka/actor/IO.scala b/akka-actor/src/main/scala/akka/actor/IO.scala index b7bfe9f0b4..b7942975d9 100644 --- a/akka-actor/src/main/scala/akka/actor/IO.scala +++ b/akka-actor/src/main/scala/akka/actor/IO.scala @@ -134,6 +134,12 @@ object IO { */ case class Listen(server: ServerHandle, address: SocketAddress) extends IOMessage + /** + * Message from an [[akka.actor.IOManager]] that the ServerSocketChannel is + * now listening for connections. + * + * No action is required by the receiving [[akka.actor.Actor]]. + */ case class Listening(server: ServerHandle, address: SocketAddress) extends IOMessage /** @@ -161,8 +167,7 @@ object IO { * Message from an [[akka.actor.IOManager]] that the SocketChannel has * successfully connected. * - * No action is required by the receiving [[akka.actor.Actor]], although - * the message still needs to be in it's receive method. + * No action is required by the receiving [[akka.actor.Actor]]. */ case class Connected(socket: SocketHandle, address: SocketAddress) extends IOMessage @@ -178,8 +183,7 @@ object IO { * optionally contain the Exception that caused the Channel to close, if * applicable. * - * No action is required by the receiving [[akka.actor.Actor]], although - * the message still needs to be in it's receive method. + * No action is required by the receiving [[akka.actor.Actor]]. */ case class Closed(handle: Handle, cause: Option[Exception]) extends IOMessage From 4c6f9b86d13b48360edd8e8a8214de776ac6498c Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Mon, 23 Jan 2012 23:44:41 -0700 Subject: [PATCH 119/152] More IO docs --- akka-docs/scala/io.rst | 81 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 81 insertions(+) diff --git a/akka-docs/scala/io.rst b/akka-docs/scala/io.rst index 2379902716..81c4eaf2da 100644 --- a/akka-docs/scala/io.rst +++ b/akka-docs/scala/io.rst @@ -12,6 +12,87 @@ Introduction This documentation is in progress. More to come. +Components +---------- + +ByteString +^^^^^^^^^^ + +A primary goal of Akka's IO module is to only communicate between actors with immutable objects. When dealing with network IO on the jvm ``Array[Byte]`` and ``ByteBuffer`` are commonly used to represent collections of ``Byte``\s, but they are mutable. Scala's collection library also lacks a suitably efficient immutable collection for ``Byte``\s. Being able to safely and efficiently move ``Byte``\s around is very important for this IO module, so ``ByteString`` was developed. + +``ByteString`` is a `Rope-like `_ data structure that is immutable and efficient. When 2 ``ByteString``\s are concatenated together they are both stored within the resulting ``ByteString`` instead of copying both to a new ``Array``. Operations such as ``drop`` and ``take`` return ``ByteString``\s that still reference the original ``Array``, but just change the offset and length that is visible. Great care has also been taken to make sure that the internal ``Array`` cannot be modified. Whenever a potentially unsafe ``Array`` is used to create a new ``ByteString`` a defensive copy is created. + +``ByteString`` inherits all methods from ``IndexedSeq``, and it also has some new ones: + +copyToBuffer(buffer: ByteBuffer): Int + Copy as many bytes as possible to a ``ByteBuffer``, starting from it's current position. This method will not overflow the buffer. It returns the number of bytes copied. + +compact: ByteString + Creates a new ``ByteString`` with all contents compacted into a single byte array. If the contents of this ``ByteString`` are already compacted it will return itself unchanged. + +asByteBuffer: ByteBuffer + If possible this will return a read-only ``ByteBuffer`` that wraps the internal byte array. If this ``ByteString`` contains more then one byte array then this method will return the result of ``toByteBuffer``. + +toByteBuffer: ByteBuffer + Creates a new ByteBuffer with a copy of all bytes contained in this ``ByteString``. + +decodeString(charset: String): String + Decodes this ``ByteString`` using a charset to produce a ``String``. + +utf8String: String + Decodes this ``ByteString`` as a UTF-8 encoded String. + +There are also several factory methods in the ``ByteString`` companion object to assist in creating a new ``ByteString``. The ``apply`` method accepts ``Array[Byte]``, ``Byte*``, ``ByteBuffer``, ``String``, as well as a ``String`` with a charset. There is also ``fromArray(array, offset, length)`` for creating a ``ByteString`` using only part of an ``Array[Byte]``. + +Finally, there is a ``ByteStringBuilder`` to build up a ``ByteString`` using Scala's mutable ``Builder`` concept. It can be especially useful when many ``ByteString``\s need to be concatenated in a performance critical section of a program. + +IO.Handle +^^^^^^^^^ + +``IO.Handle`` is an immutable reference to a Java NIO ``Channel``. Passing mutable ``Channel``\s between ``Actor``\s could lead to unsafe behavior, so instead subclasses of the ``IO.Handle`` trait are used. Currently there are 2 concrete subclasses: ``IO.SocketHandle`` (representing a ``SocketChannel``) and ``IO.ServerHandle`` (representing a ``ServerSocketChannel``). + +IOManager +^^^^^^^^^ + +The ``IOManager`` takes care of the low level IO details. Each ``ActorSystem`` has it's own ``IOManager``, which can be accessed calling ``IOManager(system: ActorSystem)``. ``Actor``\s communicate with the ``IOManager`` with specific messages. The messages sent from an ``Actor`` to the ``IOManager`` are created and sent from certain methods: + +IOManager(system).connect(address: SocketAddress): IO.SocketHandle + Opens a ``SocketChannel`` and connects to an address. Can also use ``connect(host: String, port: Int)``. + +IOManager(system).listen(address: SocketAddress): IO.ServerHandle + Opens a ``ServerSocketChannel`` and listens on an address. Can also use ``listen(host: String, port: Int)``. + +socketHandle.write(bytes: ByteString) + Write to the ``SocketChannel``. + +serverHandle.accept(): IO.SocketHandle + Accepts an incoming connection, and returns the ``IO.SocketHandle`` for the new connection. + +handle.close() + Closes the ``Channel``. + +Messages that the ``IOManager`` can send to an ``Actor`` are: + +IO.Listening(server: IO.ServerHandle, address: SocketAddress) + Sent when a ``ServerSocketChannel`` is created. If port 0 (random port) was requested then the address returned here will contain the actual port. + +IO.Connected(socket: IO.SocketHandle, address: SocketAddress) + Sent after a ``SocketChannel`` has successfully connected. + +IO.NewClient(server: IO.ServerHandle) + Sent when a new client has connected to a ``ServerSocketChannel``. The ``accept`` method must be called on the ``IO.ServerHandle`` in order to get the ``IO.SocketHandle`` to communicate to the new client. + +IO.Read(handle: IO.ReadHandle, bytes: ByteString) + Sent when bytes have been read from a ``SocketChannel``. The handle is a ``IO.ReadHandle``, which is a superclass of ``IO.SocketHandle``. + +IO.Closed(handle: IO.Handle, cause: Option[Exception]) + Sent when a ``Channel`` has closed. If an ``Exception`` was thrown due to this ``Channel`` closing it will be contained here. + +IO.Iteratee +^^^^^^^^^^^ + +See example below. + Examples -------- From abc072ef0a05703b1fc6a9a00c7d1c92aa1d8311 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 24 Jan 2012 08:37:01 +0100 Subject: [PATCH 120/152] Improved API of OneForOneStrategy and AllForOneStrategy. See #1714 * withinTimeRange: Duration * Removed need for Options in API --- .../ActorFireForgetRequestReplySpec.scala | 3 +- .../scala/akka/actor/ActorLifeCycleSpec.scala | 6 +- .../test/scala/akka/actor/ActorRefSpec.scala | 2 +- .../scala/akka/actor/DeathWatchSpec.scala | 4 +- .../scala/akka/actor/FSMTransitionSpec.scala | 4 +- .../akka/actor/RestartStrategySpec.scala | 13 ++- .../test/scala/akka/actor/SchedulerSpec.scala | 6 +- .../akka/actor/SupervisorHierarchySpec.scala | 9 +- .../scala/akka/actor/SupervisorMiscSpec.scala | 4 +- .../scala/akka/actor/SupervisorSpec.scala | 41 ++++---- .../scala/akka/actor/SupervisorTreeSpec.scala | 3 +- .../test/scala/akka/actor/Ticket669Spec.scala | 6 +- .../scala/akka/event/LoggingReceiveSpec.scala | 3 +- .../src/main/scala/akka/actor/Actor.scala | 2 +- .../main/scala/akka/actor/FaultHandling.scala | 97 ++++++++++--------- .../main/scala/akka/actor/UntypedActor.scala | 2 +- .../docs/actor/FaultHandlingTestBase.java | 4 +- .../project/migration-guide-1.3.x-2.0.x.rst | 2 +- .../docs/actor/FaultHandlingDocSpec.scala | 6 +- .../scala/akka/testkit/TestActorRefSpec.scala | 3 +- 20 files changed, 122 insertions(+), 98 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala index c9f1725692..97ebdbaa1e 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala @@ -81,7 +81,8 @@ class ActorFireForgetRequestReplySpec extends AkkaSpec with BeforeAndAfterEach w "should shutdown crashed temporary actor" in { filterEvents(EventFilter[Exception]("Expected exception")) { - val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), Some(0))))) + val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), + maxNrOfRetries = 0)))) val actor = Await.result((supervisor ? Props[CrashingActor]).mapTo[ActorRef], timeout.duration) actor.isTerminated must be(false) actor ! "Die" diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala index d2c8a4bd47..fe30c2b39b 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala @@ -36,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 supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), Some(3))))) + val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), maxNrOfRetries = 3)))) val gen = new AtomicInteger(0) val restarterProps = Props(new LifeCycleTestActor(testActor, id, gen) { override def preRestart(reason: Throwable, message: Option[Any]) { report("preRestart") } @@ -70,7 +70,7 @@ class ActorLifeCycleSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitS "default for preRestart and postRestart is to call postStop and preStart respectively" in { filterException[ActorKilledException] { val id = newUuid().toString - val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), Some(3))))) + val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), maxNrOfRetries = 3)))) val gen = new AtomicInteger(0) val restarterProps = Props(new LifeCycleTestActor(testActor, id, gen)) val restarter = Await.result((supervisor ? restarterProps).mapTo[ActorRef], timeout.duration) @@ -100,7 +100,7 @@ class ActorLifeCycleSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitS "not invoke preRestart and postRestart when never restarted using OneForOneStrategy" in { val id = newUuid().toString - val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), Some(3))))) + val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), maxNrOfRetries = 3)))) val gen = new AtomicInteger(0) val props = Props(new LifeCycleTestActor(testActor, id, gen)) val a = Await.result((supervisor ? props).mapTo[ActorRef], timeout.duration) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala index 1f5120decd..5158de2262 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala @@ -376,7 +376,7 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout { val boss = system.actorOf(Props(new Actor { - override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), 2, 1000) + override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), maxNrOfRetries = 2, withinTimeRange = 1 second) val ref = context.actorOf( Props(new Actor { diff --git a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala index 44d3daa9e2..921e57d2a4 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala @@ -95,7 +95,7 @@ trait DeathWatchSpec { this: AkkaSpec with ImplicitSender with DefaultTimeout "notify with a Terminated message once when an Actor is stopped but not when restarted" in { filterException[ActorKilledException] { - val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), Some(2))))) + val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), maxNrOfRetries = 2)))) val terminalProps = Props(context ⇒ { case x ⇒ context.sender ! x }) val terminal = Await.result((supervisor ? terminalProps).mapTo[ActorRef], timeout.duration) @@ -116,7 +116,7 @@ trait DeathWatchSpec { this: AkkaSpec with ImplicitSender with DefaultTimeout "fail a monitor which does not handle Terminated()" in { filterEvents(EventFilter[ActorKilledException](), EventFilter[DeathPactException]()) { case class FF(fail: Failed) - val strategy = new OneForOneStrategy(SupervisorStrategy.makeDecider(List(classOf[Exception])), Some(0)) { + val strategy = new OneForOneStrategy(SupervisorStrategy.makeDecider(List(classOf[Exception])), maxNrOfRetries = 0) { override def handleFailure(context: ActorContext, child: ActorRef, cause: Throwable, stats: ChildRestartStats, children: Iterable[ChildRestartStats]) = { testActor.tell(FF(Failed(cause)), child) super.handleFailure(context, child, cause, stats, children) diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala index 36258883e9..d0c270070e 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala @@ -5,8 +5,8 @@ package akka.actor import akka.testkit._ import akka.util.duration._ - import FSM._ +import akka.util.Duration object FSMTransitionSpec { @@ -72,7 +72,7 @@ class FSMTransitionSpec extends AkkaSpec with ImplicitSender { val fsm = system.actorOf(Props(new MyFSM(testActor))) val sup = system.actorOf(Props(new Actor { context.watch(fsm) - override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), None, None) + override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), Duration.Inf) def receive = { case _ ⇒ } })) diff --git a/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala index beeb243ce8..17750dd693 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala @@ -14,6 +14,7 @@ import akka.testkit.AkkaSpec import akka.testkit.DefaultTimeout import akka.testkit.TestLatch import akka.util.duration._ +import akka.util.Duration import akka.pattern.ask @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @@ -29,7 +30,8 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout { "A RestartStrategy" must { "ensure that slave stays dead after max restarts within time range" in { - val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Throwable]), 2, 1000)))) + val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Throwable]), + maxNrOfRetries = 2, withinTimeRange = 1 second)))) val restartLatch = new TestLatch val secondRestartLatch = new TestLatch @@ -75,7 +77,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout { } "ensure that slave is immortal without max restarts and time range" in { - val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Throwable]), None, None)))) + val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Throwable]), Duration.Inf)))) val countDownLatch = new TestLatch(100) @@ -97,7 +99,8 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout { } "ensure that slave restarts after number of crashes not within time range" in { - val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Throwable]), 2, 500)))) + val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Throwable]), + maxNrOfRetries = 2, withinTimeRange = 500 millis)))) val restartLatch = new TestLatch val secondRestartLatch = new TestLatch @@ -154,7 +157,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout { } "ensure that slave is not restarted after max retries" in { - val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Throwable]), Some(2), None)))) + val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Throwable]), maxNrOfRetries = 2)))) val restartLatch = new TestLatch val secondRestartLatch = new TestLatch @@ -209,7 +212,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout { val countDownLatch = new TestLatch(2) val boss = system.actorOf(Props(new Actor { - override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), None, Some(1000)) + override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), withinTimeRange = 1 second) def receive = { case p: Props ⇒ sender ! context.watch(context.actorOf(p)) case t: Terminated ⇒ maxNoOfRestartsLatch.open() diff --git a/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala index a07af9d2eb..ea7522f30d 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala @@ -134,7 +134,7 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout val restartLatch = new TestLatch val pingLatch = new TestLatch(6) - val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), 3, 1000)))) + val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), 3, 1 second)))) val props = Props(new Actor { def receive = { case Ping ⇒ pingLatch.countDown() @@ -165,8 +165,8 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout def receive = { case Msg(ts) ⇒ val now = System.nanoTime - // Make sure that no message has been dispatched before the scheduled time (10ms = 10000000ns) has occurred - if (now - ts < 10000000) throw new RuntimeException("Interval is too small: " + (now - ts)) + // Make sure that no message has been dispatched before the scheduled time (10ms) has occurred + if (now - ts < 10.millis.toNanos) throw new RuntimeException("Interval is too small: " + (now - ts)) ticks.countDown() } })) diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala index 60107b9754..185651b278 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala @@ -5,10 +5,11 @@ package akka.actor import akka.testkit._ - import java.util.concurrent.{ TimeUnit, CountDownLatch } import akka.dispatch.Await import akka.pattern.ask +import akka.util.Duration +import akka.util.duration._ object SupervisorHierarchySpec { class FireWorkerException(msg: String) extends Exception(msg) @@ -39,9 +40,9 @@ class SupervisorHierarchySpec extends AkkaSpec with DefaultTimeout { "restart manager and workers in AllForOne" in { val countDown = new CountDownLatch(4) - val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), None, None)))) + val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), Duration.Inf)))) - val managerProps = Props(new CountDownActor(countDown, AllForOneStrategy(List(), None, None))) + val managerProps = Props(new CountDownActor(countDown, AllForOneStrategy(List(), Duration.Inf))) val manager = Await.result((boss ? managerProps).mapTo[ActorRef], timeout.duration) val workerProps = Props(new CountDownActor(countDown, SupervisorStrategy.defaultStrategy)) @@ -61,7 +62,7 @@ class SupervisorHierarchySpec extends AkkaSpec with DefaultTimeout { val countDownMessages = new CountDownLatch(1) val countDownMax = new CountDownLatch(1) val boss = system.actorOf(Props(new Actor { - override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), 1, 5000) + override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), maxNrOfRetries = 1, withinTimeRange = 5 seconds) val crasher = context.watch(context.actorOf(Props(new CountDownActor(countDownMessages, SupervisorStrategy.defaultStrategy)))) diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala index de617c8db2..7eb5007152 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala @@ -9,6 +9,7 @@ import java.util.concurrent.{ TimeUnit, CountDownLatch } import akka.testkit.AkkaSpec import akka.testkit.DefaultTimeout import akka.pattern.ask +import akka.util.duration._ object SupervisorMiscSpec { val config = """ @@ -29,7 +30,8 @@ class SupervisorMiscSpec extends AkkaSpec(SupervisorMiscSpec.config) with Defaul filterEvents(EventFilter[Exception]("Kill")) { val countDownLatch = new CountDownLatch(4) - val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), 3, 5000)))) + val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), + maxNrOfRetries = 3, withinTimeRange = 5 seconds)))) val workerProps = Props(new Actor { override def postRestart(cause: Throwable) { countDownLatch.countDown() } diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala index 6b96038000..630da50f6d 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala @@ -14,7 +14,7 @@ import akka.dispatch.Await import akka.pattern.ask object SupervisorSpec { - val Timeout = 5 seconds + val Timeout = 5.seconds case object DieReply @@ -54,7 +54,7 @@ object SupervisorSpec { var s: ActorRef = _ - override val supervisorStrategy = OneForOneStrategy(List(classOf[Exception]), Some(0)) + override val supervisorStrategy = OneForOneStrategy(List(classOf[Exception]), maxNrOfRetries = 0) def receive = { case Die ⇒ temp forward Die @@ -69,7 +69,7 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende import SupervisorSpec._ - val TimeoutMillis = Timeout.dilated.toMillis.toInt + val DilatedTimeout = Timeout.dilated // ===================================================== // Creating actors and supervisors @@ -78,45 +78,51 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende private def child(supervisor: ActorRef, props: Props): ActorRef = Await.result((supervisor ? props).mapTo[ActorRef], timeout.duration) def temporaryActorAllForOne = { - val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), Some(0))))) + val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), maxNrOfRetries = 0)))) val temporaryActor = child(supervisor, Props(new PingPongActor(testActor))) (temporaryActor, supervisor) } def singleActorAllForOne = { - val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis)))) + val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), + maxNrOfRetries = 3, withinTimeRange = DilatedTimeout)))) val pingpong = child(supervisor, Props(new PingPongActor(testActor))) (pingpong, supervisor) } def singleActorOneForOne = { - val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis)))) + val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), + maxNrOfRetries = 3, withinTimeRange = DilatedTimeout)))) val pingpong = child(supervisor, Props(new PingPongActor(testActor))) (pingpong, supervisor) } def multipleActorsAllForOne = { - val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis)))) + val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), + maxNrOfRetries = 3, withinTimeRange = DilatedTimeout)))) val pingpong1, pingpong2, pingpong3 = child(supervisor, Props(new PingPongActor(testActor))) (pingpong1, pingpong2, pingpong3, supervisor) } def multipleActorsOneForOne = { - val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis)))) + val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), + maxNrOfRetries = 3, withinTimeRange = DilatedTimeout)))) val pingpong1, pingpong2, pingpong3 = child(supervisor, Props(new PingPongActor(testActor))) (pingpong1, pingpong2, pingpong3, supervisor) } def nestedSupervisorsAllForOne = { - val topSupervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis)))) + val topSupervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), + maxNrOfRetries = 3, withinTimeRange = DilatedTimeout)))) val pingpong1 = child(topSupervisor, Props(new PingPongActor(testActor))) - val middleSupervisor = child(topSupervisor, Props(new Supervisor(AllForOneStrategy(Nil, 3, TimeoutMillis)))) + val middleSupervisor = child(topSupervisor, Props(new Supervisor(AllForOneStrategy(Nil, + maxNrOfRetries = 3, withinTimeRange = DilatedTimeout)))) val pingpong2, pingpong3 = child(middleSupervisor, Props(new PingPongActor(testActor))) (pingpong1, pingpong2, pingpong3, topSupervisor) @@ -131,14 +137,14 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende } def ping(pingPongActor: ActorRef) = { - Await.result(pingPongActor.?(Ping)(TimeoutMillis), TimeoutMillis millis) must be === PongMessage + Await.result(pingPongActor.?(Ping)(DilatedTimeout), DilatedTimeout) must be === PongMessage expectMsg(Timeout, PingMessage) } def kill(pingPongActor: ActorRef) = { - val result = (pingPongActor.?(DieReply)(TimeoutMillis)) + val result = (pingPongActor.?(DieReply)(DilatedTimeout)) expectMsg(Timeout, ExceptionMessage) - intercept[RuntimeException] { Await.result(result, TimeoutMillis millis) } + intercept[RuntimeException] { Await.result(result, DilatedTimeout) } } "A supervisor" must { @@ -154,7 +160,7 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende "not restart temporary actor" in { val (temporaryActor, _) = temporaryActorAllForOne - intercept[RuntimeException] { Await.result(temporaryActor.?(DieReply)(TimeoutMillis), TimeoutMillis millis) } + intercept[RuntimeException] { Await.result(temporaryActor.?(DieReply)(DilatedTimeout), DilatedTimeout) } expectNoMsg(1 second) } @@ -280,7 +286,8 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende "must attempt restart when exception during restart" in { val inits = new AtomicInteger(0) - val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(classOf[Exception] :: Nil, 3, 10000)))) + val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(classOf[Exception] :: Nil, + maxNrOfRetries = 3, withinTimeRange = 10 seconds)))) val dyingProps = Props(new Actor { inits.incrementAndGet @@ -300,11 +307,11 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende filterEvents(EventFilter[RuntimeException]("Expected", occurrences = 1), EventFilter[IllegalStateException]("error while creating actor", occurrences = 1)) { intercept[RuntimeException] { - Await.result(dyingActor.?(DieReply)(TimeoutMillis), TimeoutMillis millis) + Await.result(dyingActor.?(DieReply)(DilatedTimeout), DilatedTimeout) } } - Await.result(dyingActor.?(Ping)(TimeoutMillis), TimeoutMillis millis) must be === PongMessage + Await.result(dyingActor.?(Ping)(DilatedTimeout), DilatedTimeout) must be === PongMessage inits.get must be(3) diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala index 04c1292d15..b9e3ff4a20 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala @@ -23,7 +23,8 @@ class SupervisorTreeSpec extends AkkaSpec with ImplicitSender with DefaultTimeou EventFilter[ActorKilledException](occurrences = 1) intercept { within(5 seconds) { val p = Props(new Actor { - override val supervisorStrategy = OneForOneStrategy(List(classOf[Exception]), 3, 1000) + override val supervisorStrategy = OneForOneStrategy(List(classOf[Exception]), + maxNrOfRetries = 3, withinTimeRange = 1 second) def receive = { case p: Props ⇒ sender ! context.actorOf(p) } diff --git a/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala b/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala index 38292decde..b43ffc6d0c 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala @@ -12,6 +12,7 @@ import akka.testkit.ImplicitSender import akka.testkit.DefaultTimeout import akka.dispatch.Await import akka.pattern.ask +import akka.util.duration._ @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class Ticket669Spec extends AkkaSpec with BeforeAndAfterAll with ImplicitSender with DefaultTimeout { @@ -25,7 +26,7 @@ class Ticket669Spec extends AkkaSpec with BeforeAndAfterAll with ImplicitSender "A supervised actor with lifecycle PERMANENT" should { "be able to reply on failure during preRestart" in { filterEvents(EventFilter[Exception]("test", occurrences = 1)) { - val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), 5, 10000)))) + val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), 5, 10 seconds)))) val supervised = Await.result((supervisor ? Props[Supervised]).mapTo[ActorRef], timeout.duration) supervised.!("test")(testActor) @@ -36,7 +37,8 @@ class Ticket669Spec extends AkkaSpec with BeforeAndAfterAll with ImplicitSender "be able to reply on failure during postStop" in { filterEvents(EventFilter[Exception]("test", occurrences = 1)) { - val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), Some(0), None)))) + val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), + maxNrOfRetries = 0)))) val supervised = Await.result((supervisor ? Props[Supervised]).mapTo[ActorRef], timeout.duration) supervised.!("test")(testActor) diff --git a/akka-actor-tests/src/test/scala/akka/event/LoggingReceiveSpec.scala b/akka-actor-tests/src/test/scala/akka/event/LoggingReceiveSpec.scala index cf620bf0fc..1857c0938b 100644 --- a/akka-actor-tests/src/test/scala/akka/event/LoggingReceiveSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/event/LoggingReceiveSpec.scala @@ -15,7 +15,8 @@ import akka.actor._ object LoggingReceiveSpec { class TestLogActor extends Actor { - override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), 5, 5000) + override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), + maxNrOfRetries = 5, withinTimeRange = 5 seconds) def receive = { case _ ⇒ } } } diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index 40b590aa51..86e9cb816d 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -147,7 +147,7 @@ object Actor { * case _: NullPointerException ⇒ Restart * case _: IllegalArgumentException ⇒ Stop * case _: Exception ⇒ Escalate - * }: Decider, maxNrOfRetries = Some(10), withinTimeRange = Some(60000)) + * }: Decider, maxNrOfRetries = 10, withinTimeRange = 1 minute)) * * def receive = { * // directly calculated reply diff --git a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala index e7ae22f57f..79166a01eb 100644 --- a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala @@ -8,6 +8,7 @@ import scala.annotation.tailrec import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import java.lang.{ Iterable ⇒ JIterable } +import akka.util.Duration case class ChildRestartStats(val child: ActorRef, var maxNrOfRetriesCount: Int = 0, var restartTimeWindowStartNanos: Long = 0L) { @@ -95,6 +96,13 @@ object SupervisorStrategy { */ def escalate = Escalate + /** + * When supervisorStrategy is not specified for an actor this + * is used by default. The child will be stopped when + * [[akka.ActorInitializationException]] or [[akka.ActorKilledException]] + * is thrown. It will be restarted for other `Exception` types. + * The error is escalated if it's a `Throwable`, i.e. `Error`. + */ final val defaultStrategy: SupervisorStrategy = { def defaultDecider: Decider = { case _: ActorInitializationException ⇒ Stop @@ -102,7 +110,7 @@ object SupervisorStrategy { case _: Exception ⇒ Restart case _ ⇒ Escalate } - OneForOneStrategy(defaultDecider, None, None) + OneForOneStrategy(defaultDecider) } type Decider = PartialFunction[Throwable, Action] @@ -120,14 +128,14 @@ object SupervisorStrategy { * Backwards compatible Decider builder which just checks whether one of * the given Throwables matches the cause and restarts, otherwise escalates. */ - def makeDecider(trapExit: List[Class[_ <: Throwable]]): Decider = + def makeDecider(trapExit: Seq[Class[_ <: Throwable]]): Decider = { case x ⇒ if (trapExit exists (_ isInstance x)) Restart else Escalate } /** * Backwards compatible Decider builder which just checks whether one of * the given Throwables matches the cause and restarts, otherwise escalates. */ - def makeDecider(trapExit: JIterable[Class[_ <: Throwable]]): Decider = makeDecider(trapExit.toList) + def makeDecider(trapExit: JIterable[Class[_ <: Throwable]]): Decider = makeDecider(trapExit.toSeq) /** * Decider builder for Iterables of cause-action pairs, e.g. a map obtained @@ -156,6 +164,11 @@ object SupervisorStrategy { } buf } + + 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) } abstract class SupervisorStrategy { @@ -199,46 +212,41 @@ abstract class SupervisorStrategy { } object AllForOneStrategy { - def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int): AllForOneStrategy = - new AllForOneStrategy(SupervisorStrategy.makeDecider(trapExit), - if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange)) - def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): AllForOneStrategy = + def apply(trapExit: Seq[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Duration): AllForOneStrategy = new AllForOneStrategy(SupervisorStrategy.makeDecider(trapExit), maxNrOfRetries, withinTimeRange) - def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Option[Int]): AllForOneStrategy = - new AllForOneStrategy(SupervisorStrategy.makeDecider(trapExit), maxNrOfRetries, None) + def apply(trapExit: Seq[Class[_ <: Throwable]], maxNrOfRetries: Int): AllForOneStrategy = + apply(trapExit, maxNrOfRetries, Duration.Inf) + def apply(trapExit: Seq[Class[_ <: Throwable]], withinTimeRange: Duration): AllForOneStrategy = + apply(trapExit, -1, withinTimeRange) } /** * Restart all actors linked to the same supervisor when one fails, * trapExit = which Throwables should be intercepted * maxNrOfRetries = the number of times an actor is allowed to be restarted - * withinTimeRange = millisecond time window for maxNrOfRetries, negative means no window + * withinTimeRange = duration of the time window for maxNrOfRetries, Duration.Inf means no window */ case class AllForOneStrategy(decider: SupervisorStrategy.Decider, - maxNrOfRetries: Option[Int] = None, - withinTimeRange: Option[Int] = None) extends SupervisorStrategy { + maxNrOfRetries: Int = -1, + withinTimeRange: Duration = Duration.Inf) extends SupervisorStrategy { - def this(decider: SupervisorStrategy.JDecider, maxNrOfRetries: Int, withinTimeRange: Int) = - this(SupervisorStrategy.makeDecider(decider), - if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), - if (withinTimeRange < 0) None else Some(withinTimeRange)) + def this(decider: SupervisorStrategy.JDecider, maxNrOfRetries: Int, withinTimeRange: Duration) = + this(SupervisorStrategy.makeDecider(decider), maxNrOfRetries, withinTimeRange) - def this(trapExit: JIterable[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) = - this(SupervisorStrategy.makeDecider(trapExit), - if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), - if (withinTimeRange < 0) None else Some(withinTimeRange)) + def this(trapExit: JIterable[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Duration) = + this(SupervisorStrategy.makeDecider(trapExit), maxNrOfRetries, withinTimeRange) - def this(trapExit: Array[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) = - this(SupervisorStrategy.makeDecider(trapExit), - if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), - if (withinTimeRange < 0) None else Some(withinTimeRange)) + def this(trapExit: Array[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Duration) = + this(SupervisorStrategy.makeDecider(trapExit), maxNrOfRetries, withinTimeRange) /* * this is a performance optimization to avoid re-allocating the pairs upon * every call to requestRestartPermission, assuming that strategies are shared * across actors and thus this field does not take up much space */ - val retriesWindow = (maxNrOfRetries, withinTimeRange) + private val retriesWindow = ( + SupervisorStrategy.maxNrOfRetriesOption(maxNrOfRetries), + SupervisorStrategy.withinTimeRangeOption(withinTimeRange).map(_.toMillis.toInt)) def handleChildTerminated(context: ActorContext, child: ActorRef, children: Iterable[ActorRef]): Unit = { children foreach (context.stop(_)) @@ -256,46 +264,41 @@ case class AllForOneStrategy(decider: SupervisorStrategy.Decider, } object OneForOneStrategy { - def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int): OneForOneStrategy = - new OneForOneStrategy(SupervisorStrategy.makeDecider(trapExit), - if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange)) - def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): OneForOneStrategy = + def apply(trapExit: Seq[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Duration): OneForOneStrategy = new OneForOneStrategy(SupervisorStrategy.makeDecider(trapExit), maxNrOfRetries, withinTimeRange) - def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Option[Int]): OneForOneStrategy = - new OneForOneStrategy(SupervisorStrategy.makeDecider(trapExit), maxNrOfRetries, None) + def apply(trapExit: Seq[Class[_ <: Throwable]], maxNrOfRetries: Int): OneForOneStrategy = + apply(trapExit, maxNrOfRetries, Duration.Inf) + def apply(trapExit: Seq[Class[_ <: Throwable]], withinTimeRange: Duration): OneForOneStrategy = + apply(trapExit, -1, withinTimeRange) } /** * Restart an actor when it fails * trapExit = which Throwables should be intercepted * maxNrOfRetries = the number of times an actor is allowed to be restarted - * withinTimeRange = millisecond time window for maxNrOfRetries, negative means no window + * withinTimeRange = duration of the time window for maxNrOfRetries, Duration.Inf means no window */ case class OneForOneStrategy(decider: SupervisorStrategy.Decider, - maxNrOfRetries: Option[Int] = None, - withinTimeRange: Option[Int] = None) extends SupervisorStrategy { + maxNrOfRetries: Int = -1, + withinTimeRange: Duration = Duration.Inf) extends SupervisorStrategy { - def this(decider: SupervisorStrategy.JDecider, maxNrOfRetries: Int, withinTimeRange: Int) = - this(SupervisorStrategy.makeDecider(decider), - if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), - if (withinTimeRange < 0) None else Some(withinTimeRange)) + def this(decider: SupervisorStrategy.JDecider, maxNrOfRetries: Int, withinTimeRange: Duration) = + this(SupervisorStrategy.makeDecider(decider), maxNrOfRetries, withinTimeRange) - def this(trapExit: JIterable[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) = - this(SupervisorStrategy.makeDecider(trapExit), - if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), - if (withinTimeRange < 0) None else Some(withinTimeRange)) + def this(trapExit: JIterable[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Duration) = + this(SupervisorStrategy.makeDecider(trapExit), maxNrOfRetries, withinTimeRange) - def this(trapExit: Array[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) = - this(SupervisorStrategy.makeDecider(trapExit), - if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), - if (withinTimeRange < 0) None else Some(withinTimeRange)) + def this(trapExit: Array[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Duration) = + this(SupervisorStrategy.makeDecider(trapExit), maxNrOfRetries, withinTimeRange) /* * this is a performance optimization to avoid re-allocating the pairs upon * every call to requestRestartPermission, assuming that strategies are shared * across actors and thus this field does not take up much space */ - val retriesWindow = (maxNrOfRetries, withinTimeRange) + private val retriesWindow = ( + SupervisorStrategy.maxNrOfRetriesOption(maxNrOfRetries), + SupervisorStrategy.withinTimeRangeOption(withinTimeRange).map(_.toMillis.toInt)) def handleChildTerminated(context: ActorContext, child: ActorRef, children: Iterable[ActorRef]): Unit = {} diff --git a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala index ba677b750b..46afd447d6 100644 --- a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala @@ -50,7 +50,7 @@ import akka.dispatch.{ MessageDispatcher, Promise } * return escalate(); * } * } - * }, 10, 60000); + * }, 10, Duration.parse("1 minute"); * * @Override * public SupervisorStrategy supervisorStrategy() { diff --git a/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java b/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java index e33644409d..09be1aaaca 100644 --- a/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java +++ b/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java @@ -52,7 +52,7 @@ public class FaultHandlingTestBase { return escalate(); } } - }, 10, 60000); + }, 10, Duration.parse("1 minute")); @Override public SupervisorStrategy supervisorStrategy() { @@ -89,7 +89,7 @@ public class FaultHandlingTestBase { return escalate(); } } - }, 10, 60000); + }, 10, Duration.parse("1 minute")); @Override public SupervisorStrategy supervisorStrategy() { diff --git a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst index 57a19fb393..a8b17fa2de 100644 --- a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst +++ b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst @@ -484,7 +484,7 @@ v2.0:: case _: NullPointerException ⇒ Restart case _: IllegalArgumentException ⇒ Stop case _: Exception ⇒ Escalate - }: Decider, maxNrOfRetries = Some(10), withinTimeRange = Some(60000)) + }: Decider, maxNrOfRetries = 10, withinTimeRange = 1 minute) def receive = { case x => diff --git a/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala b/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala index 22790d2110..5e6b15078c 100644 --- a/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala @@ -20,13 +20,14 @@ object FaultHandlingDocSpec { //#strategy import akka.actor.OneForOneStrategy import akka.actor.SupervisorStrategy._ + import akka.util.duration._ override val supervisorStrategy = OneForOneStrategy({ case _: ArithmeticException ⇒ Resume case _: NullPointerException ⇒ Restart case _: IllegalArgumentException ⇒ Stop case _: Exception ⇒ Escalate - }: Decider, maxNrOfRetries = Some(10), withinTimeRange = Some(60000)) + }: Decider, maxNrOfRetries = 10, withinTimeRange = 1 minute) //#strategy def receive = { @@ -40,13 +41,14 @@ object FaultHandlingDocSpec { //#strategy2 import akka.actor.OneForOneStrategy import akka.actor.SupervisorStrategy._ + import akka.util.duration._ override val supervisorStrategy = OneForOneStrategy({ case _: ArithmeticException ⇒ Resume case _: NullPointerException ⇒ Restart case _: IllegalArgumentException ⇒ Stop case _: Exception ⇒ Escalate - }: Decider, maxNrOfRetries = Some(10), withinTimeRange = Some(60000)) + }: Decider, maxNrOfRetries = 10, withinTimeRange = 1 minute) //#strategy2 def receive = { diff --git a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala index eee9318c02..48a62b21eb 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala @@ -182,7 +182,8 @@ class TestActorRefSpec extends AkkaSpec("disp1.type=Dispatcher") with BeforeAndA override def postRestart(reason: Throwable) { counter -= 1 } }), self, "child") - override def supervisorStrategy = OneForOneStrategy(List(classOf[ActorKilledException]), 5, 1000) + override def supervisorStrategy = OneForOneStrategy(List(classOf[ActorKilledException]), + maxNrOfRetries = 5, withinTimeRange = 1 second) def receiveT = { case "sendKill" ⇒ ref ! Kill } })) From ec1433c1563f8410ba3b2e81b5d0dd7cc46d6fce Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 24 Jan 2012 09:35:00 +0100 Subject: [PATCH 121/152] code style touch up of FSM.scala --- .../src/main/scala/akka/actor/FSM.scala | 69 +++++++------------ .../src/main/scala/Buncher.scala | 2 +- 2 files changed, 24 insertions(+), 47 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/FSM.scala b/akka-actor/src/main/scala/akka/actor/FSM.scala index ee9d38efab..eb7081899c 100644 --- a/akka-actor/src/main/scala/akka/actor/FSM.scala +++ b/akka-actor/src/main/scala/akka/actor/FSM.scala @@ -193,14 +193,12 @@ trait FSM[S, D] extends Listeners { * @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) = { + protected 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) = { + 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. @@ -211,9 +209,8 @@ trait FSM[S, D] extends Listeners { */ protected final def startWith(stateName: S, stateData: D, - timeout: Timeout = None) = { + timeout: Timeout = None): Unit = currentState = FSM.State(stateName, stateData, timeout) - } /** * Produce transition to other state. Return this from a state function in @@ -222,9 +219,7 @@ trait FSM[S, D] extends Listeners { * @param nextStateName state designator for the next state * @return state transition descriptor */ - protected final def goto(nextStateName: S): State = { - FSM.State(nextStateName, currentState.stateData) - } + protected final def goto(nextStateName: S): State = FSM.State(nextStateName, currentState.stateData) /** * Produce "empty" transition descriptor. Return this from a state function @@ -232,31 +227,22 @@ trait FSM[S, D] extends Listeners { * * @return descriptor for staying in current state */ - protected final def stay(): State = { - // cannot directly use currentState because of the timeout field - goto(currentState.stateName) - } + protected 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) - } + protected 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) - } + protected 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) - } + protected final def stop(reason: Reason, stateData: D): State = stay using stateData withStopReason (reason) /** * Schedule named timer to deliver message after given delay, possibly repeating. @@ -280,12 +266,11 @@ trait FSM[S, D] extends Listeners { * 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) = { + protected[akka] def cancelTimer(name: String): Unit = if (timers contains name) { timers(name).cancel timers -= name } - } /** * Inquire whether the named timer is still active. Returns true unless the @@ -298,9 +283,7 @@ trait FSM[S, D] extends Listeners { * Set state timeout explicitly. This method can safely be used from within a * state handler. */ - protected final def setStateTimeout(state: S, timeout: Timeout) { - stateTimeouts(state) = timeout - } + protected final def setStateTimeout(state: S, timeout: Timeout): Unit = stateTimeouts(state) = timeout /** * This extractor is just convenience for matching a (S, S) pair, including a @@ -335,9 +318,7 @@ trait FSM[S, D] extends Listeners { * Multiple handlers may be installed, and every one of them will be * called, not only the first one matching. */ - protected final def onTransition(transitionHandler: TransitionHandler) { - transitionEvent :+= transitionHandler - } + protected final def onTransition(transitionHandler: TransitionHandler): Unit = transitionEvent :+= transitionHandler /** * Convenience wrapper for using a total function instead of a partial @@ -352,24 +333,20 @@ trait FSM[S, D] extends Listeners { /** * Set handler which is called upon termination of this FSM actor. */ - protected final def onTermination(terminationHandler: PartialFunction[StopEvent[S, D], Unit]) = { + protected final def onTermination(terminationHandler: PartialFunction[StopEvent[S, D], Unit]): Unit = terminateEvent = terminationHandler - } /** * Set handler which is called upon reception of unhandled messages. */ - protected final def whenUnhandled(stateFunction: StateFunction) = { + protected 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 { - makeTransition(currentState) - } + protected final def initialize: Unit = makeTransition(currentState) /** * Return current state name (i.e. object of type S) @@ -412,7 +389,7 @@ trait FSM[S, D] extends Listeners { private val stateFunctions = mutable.Map[S, StateFunction]() private val stateTimeouts = mutable.Map[S, Timeout]() - private def register(name: S, function: StateFunction, timeout: Timeout) { + private def register(name: S, function: StateFunction, timeout: Timeout): Unit = { if (stateFunctions contains name) { stateFunctions(name) = stateFunctions(name) orElse function stateTimeouts(name) = timeout orElse stateTimeouts(name) @@ -492,12 +469,12 @@ trait FSM[S, D] extends Listeners { } } - private def processMsg(value: Any, source: AnyRef) { + private def processMsg(value: Any, source: AnyRef): Unit = { val event = Event(value, currentState.stateData) processEvent(event, source) } - private[akka] def processEvent(event: Event, source: AnyRef) { + private[akka] def processEvent(event: Event, source: AnyRef): Unit = { val stateFunc = stateFunctions(currentState.stateName) val nextState = if (stateFunc isDefinedAt event) { stateFunc(event) @@ -508,7 +485,7 @@ trait FSM[S, D] extends Listeners { applyState(nextState) } - private[akka] def applyState(nextState: State) { + private[akka] def applyState(nextState: State): Unit = { nextState.stopReason match { case None ⇒ makeTransition(nextState) case _ ⇒ @@ -518,7 +495,7 @@ trait FSM[S, D] extends Listeners { } } - private[akka] def makeTransition(nextState: State) { + private[akka] def makeTransition(nextState: State): Unit = { if (!stateFunctions.contains(nextState.stateName)) { terminate(stay withStopReason Failure("Next state %s does not exist".format(nextState.stateName))) } else { @@ -539,9 +516,9 @@ trait FSM[S, D] extends Listeners { } } - override def postStop() { terminate(stay withStopReason Shutdown) } + override def postStop(): Unit = { terminate(stay withStopReason Shutdown) } - private def terminate(nextState: State) { + private def terminate(nextState: State): Unit = { if (!currentState.stopReason.isDefined) { val reason = nextState.stopReason.get reason match { @@ -598,13 +575,13 @@ trait LoggingFSM[S, D] extends FSM[S, D] { this: Actor ⇒ super.setTimer(name, msg, timeout, repeat) } - protected[akka] abstract override def cancelTimer(name: String) = { + 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) { + private[akka] abstract override def processEvent(event: Event, source: AnyRef): Unit = { if (debugEvent) { val srcstr = source match { case s: String ⇒ s diff --git a/akka-samples/akka-sample-fsm/src/main/scala/Buncher.scala b/akka-samples/akka-sample-fsm/src/main/scala/Buncher.scala index d039609a98..e13f7e6a98 100644 --- a/akka-samples/akka-sample-fsm/src/main/scala/Buncher.scala +++ b/akka-samples/akka-sample-fsm/src/main/scala/Buncher.scala @@ -64,7 +64,7 @@ abstract class GenericBuncher[A: Manifest, B](val singleTimeout: Duration, val m case Event(Stop, _) ⇒ stop } - when(Active, stateTimeout = Some(singleTimeout)) { + when(Active, stateTimeout = singleTimeout) { case Event(Msg(m), acc) ⇒ stay using merge(acc, m) case Event(StateTimeout, acc) ⇒ From 2a6b7f9b03d0e9db879c99e4bc6beccbf019680e Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 24 Jan 2012 10:35:09 +0100 Subject: [PATCH 122/152] Moved decider to separate parameter list, and implicit conversion from Seq[Throwable]. See #1714 --- .../ActorFireForgetRequestReplySpec.scala | 4 +- .../scala/akka/actor/ActorLifeCycleSpec.scala | 9 ++- .../test/scala/akka/actor/ActorRefSpec.scala | 3 +- .../scala/akka/actor/DeathWatchSpec.scala | 5 +- .../scala/akka/actor/FSMTransitionSpec.scala | 2 +- .../akka/actor/RestartStrategySpec.scala | 14 ++-- .../test/scala/akka/actor/SchedulerSpec.scala | 2 +- .../akka/actor/SupervisorHierarchySpec.scala | 7 +- .../scala/akka/actor/SupervisorMiscSpec.scala | 4 +- .../scala/akka/actor/SupervisorSpec.scala | 32 ++++---- .../scala/akka/actor/SupervisorTreeSpec.scala | 3 +- .../test/scala/akka/actor/Ticket669Spec.scala | 7 +- .../scala/akka/actor/TypedActorSpec.scala | 2 +- .../scala/akka/event/LoggingReceiveSpec.scala | 4 +- .../src/main/scala/akka/actor/Actor.scala | 4 +- .../scala/akka/actor/ActorRefProvider.scala | 2 +- .../main/scala/akka/actor/FaultHandling.scala | 79 ++++++++----------- .../main/scala/akka/actor/UntypedActor.scala | 27 ++++--- .../docs/actor/FaultHandlingTestBase.java | 58 +++++++------- .../project/migration-guide-1.3.x-2.0.x.rst | 6 +- .../docs/actor/FaultHandlingDocSpec.scala | 8 +- .../scala/akka/testkit/TestActorRefSpec.scala | 4 +- 22 files changed, 142 insertions(+), 144 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala index 97ebdbaa1e..69cf463276 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala @@ -81,8 +81,8 @@ class ActorFireForgetRequestReplySpec extends AkkaSpec with BeforeAndAfterEach w "should shutdown crashed temporary actor" in { filterEvents(EventFilter[Exception]("Expected exception")) { - val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), - maxNrOfRetries = 0)))) + val supervisor = system.actorOf(Props(new Supervisor( + OneForOneStrategy(maxNrOfRetries = 0)(List(classOf[Exception]))))) val actor = Await.result((supervisor ? Props[CrashingActor]).mapTo[ActorRef], timeout.duration) actor.isTerminated must be(false) actor ! "Die" diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala index fe30c2b39b..781b578d93 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala @@ -36,7 +36,8 @@ class ActorLifeCycleSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitS "invoke preRestart, preStart, postRestart when using OneForOneStrategy" in { filterException[ActorKilledException] { val id = newUuid().toString - val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), maxNrOfRetries = 3)))) + val supervisor = system.actorOf(Props(new Supervisor( + OneForOneStrategy(maxNrOfRetries = 3)(List(classOf[Exception]))))) val gen = new AtomicInteger(0) val restarterProps = Props(new LifeCycleTestActor(testActor, id, gen) { override def preRestart(reason: Throwable, message: Option[Any]) { report("preRestart") } @@ -70,7 +71,8 @@ class ActorLifeCycleSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitS "default for preRestart and postRestart is to call postStop and preStart respectively" in { filterException[ActorKilledException] { val id = newUuid().toString - val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), maxNrOfRetries = 3)))) + val supervisor = system.actorOf(Props(new Supervisor( + OneForOneStrategy(maxNrOfRetries = 3)(List(classOf[Exception]))))) val gen = new AtomicInteger(0) val restarterProps = Props(new LifeCycleTestActor(testActor, id, gen)) val restarter = Await.result((supervisor ? restarterProps).mapTo[ActorRef], timeout.duration) @@ -100,7 +102,8 @@ class ActorLifeCycleSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitS "not invoke preRestart and postRestart when never restarted using OneForOneStrategy" in { val id = newUuid().toString - val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), maxNrOfRetries = 3)))) + val supervisor = system.actorOf(Props(new Supervisor( + OneForOneStrategy(maxNrOfRetries = 3)(List(classOf[Exception]))))) val gen = new AtomicInteger(0) val props = Props(new LifeCycleTestActor(testActor, id, gen)) val a = Await.result((supervisor ? props).mapTo[ActorRef], timeout.duration) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala index 5158de2262..a4dbb4d1cb 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala @@ -376,7 +376,8 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout { val boss = system.actorOf(Props(new Actor { - override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), maxNrOfRetries = 2, withinTimeRange = 1 second) + override val supervisorStrategy = + OneForOneStrategy(maxNrOfRetries = 2, withinTimeRange = 1 second)(List(classOf[Throwable])) val ref = context.actorOf( Props(new Actor { diff --git a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala index 921e57d2a4..3e8ff57de5 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala @@ -95,7 +95,8 @@ trait DeathWatchSpec { this: AkkaSpec with ImplicitSender with DefaultTimeout "notify with a Terminated message once when an Actor is stopped but not when restarted" in { filterException[ActorKilledException] { - val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), maxNrOfRetries = 2)))) + val supervisor = system.actorOf(Props(new Supervisor( + OneForOneStrategy(maxNrOfRetries = 2)(List(classOf[Exception]))))) val terminalProps = Props(context ⇒ { case x ⇒ context.sender ! x }) val terminal = Await.result((supervisor ? terminalProps).mapTo[ActorRef], timeout.duration) @@ -116,7 +117,7 @@ trait DeathWatchSpec { this: AkkaSpec with ImplicitSender with DefaultTimeout "fail a monitor which does not handle Terminated()" in { filterEvents(EventFilter[ActorKilledException](), EventFilter[DeathPactException]()) { case class FF(fail: Failed) - val strategy = new OneForOneStrategy(SupervisorStrategy.makeDecider(List(classOf[Exception])), maxNrOfRetries = 0) { + val strategy = new OneForOneStrategy(maxNrOfRetries = 0)(SupervisorStrategy.makeDecider(List(classOf[Exception]))) { override def handleFailure(context: ActorContext, child: ActorRef, cause: Throwable, stats: ChildRestartStats, children: Iterable[ChildRestartStats]) = { testActor.tell(FF(Failed(cause)), child) super.handleFailure(context, child, cause, stats, children) diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala index d0c270070e..8d8fc5e725 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala @@ -72,7 +72,7 @@ class FSMTransitionSpec extends AkkaSpec with ImplicitSender { val fsm = system.actorOf(Props(new MyFSM(testActor))) val sup = system.actorOf(Props(new Actor { context.watch(fsm) - override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), Duration.Inf) + override val supervisorStrategy = OneForOneStrategy(withinTimeRange = Duration.Inf)(List(classOf[Throwable])) def receive = { case _ ⇒ } })) diff --git a/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala index 17750dd693..829ab081e0 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala @@ -30,8 +30,8 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout { "A RestartStrategy" must { "ensure that slave stays dead after max restarts within time range" in { - val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Throwable]), - maxNrOfRetries = 2, withinTimeRange = 1 second)))) + val boss = system.actorOf(Props(new Supervisor( + OneForOneStrategy(maxNrOfRetries = 2, withinTimeRange = 1 second)(List(classOf[Throwable]))))) val restartLatch = new TestLatch val secondRestartLatch = new TestLatch @@ -77,7 +77,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout { } "ensure that slave is immortal without max restarts and time range" in { - val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Throwable]), Duration.Inf)))) + val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy()(List(classOf[Throwable]))))) val countDownLatch = new TestLatch(100) @@ -99,8 +99,8 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout { } "ensure that slave restarts after number of crashes not within time range" in { - val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Throwable]), - maxNrOfRetries = 2, withinTimeRange = 500 millis)))) + val boss = system.actorOf(Props(new Supervisor( + OneForOneStrategy(maxNrOfRetries = 2, withinTimeRange = 500 millis)(List(classOf[Throwable]))))) val restartLatch = new TestLatch val secondRestartLatch = new TestLatch @@ -157,7 +157,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout { } "ensure that slave is not restarted after max retries" in { - val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Throwable]), maxNrOfRetries = 2)))) + val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(maxNrOfRetries = 2)(List(classOf[Throwable]))))) val restartLatch = new TestLatch val secondRestartLatch = new TestLatch @@ -212,7 +212,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout { val countDownLatch = new TestLatch(2) val boss = system.actorOf(Props(new Actor { - override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), withinTimeRange = 1 second) + override val supervisorStrategy = OneForOneStrategy(withinTimeRange = 1 second)(List(classOf[Throwable])) def receive = { case p: Props ⇒ sender ! context.watch(context.actorOf(p)) case t: Terminated ⇒ maxNoOfRestartsLatch.open() diff --git a/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala index ea7522f30d..d205d1a1ad 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala @@ -134,7 +134,7 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout val restartLatch = new TestLatch val pingLatch = new TestLatch(6) - val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), 3, 1 second)))) + val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(3, 1 second)(List(classOf[Exception]))))) val props = Props(new Actor { def receive = { case Ping ⇒ pingLatch.countDown() diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala index 185651b278..a04e83f39b 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala @@ -40,9 +40,9 @@ class SupervisorHierarchySpec extends AkkaSpec with DefaultTimeout { "restart manager and workers in AllForOne" in { val countDown = new CountDownLatch(4) - val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), Duration.Inf)))) + val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy()(List(classOf[Exception]))))) - val managerProps = Props(new CountDownActor(countDown, AllForOneStrategy(List(), Duration.Inf))) + val managerProps = Props(new CountDownActor(countDown, AllForOneStrategy()(List()))) val manager = Await.result((boss ? managerProps).mapTo[ActorRef], timeout.duration) val workerProps = Props(new CountDownActor(countDown, SupervisorStrategy.defaultStrategy)) @@ -62,7 +62,8 @@ class SupervisorHierarchySpec extends AkkaSpec with DefaultTimeout { val countDownMessages = new CountDownLatch(1) val countDownMax = new CountDownLatch(1) val boss = system.actorOf(Props(new Actor { - override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), maxNrOfRetries = 1, withinTimeRange = 5 seconds) + override val supervisorStrategy = + OneForOneStrategy(maxNrOfRetries = 1, withinTimeRange = 5 seconds)(List(classOf[Throwable])) val crasher = context.watch(context.actorOf(Props(new CountDownActor(countDownMessages, SupervisorStrategy.defaultStrategy)))) diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala index 7eb5007152..fccfc75d98 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala @@ -30,8 +30,8 @@ class SupervisorMiscSpec extends AkkaSpec(SupervisorMiscSpec.config) with Defaul filterEvents(EventFilter[Exception]("Kill")) { val countDownLatch = new CountDownLatch(4) - val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), - maxNrOfRetries = 3, withinTimeRange = 5 seconds)))) + val supervisor = system.actorOf(Props(new Supervisor( + OneForOneStrategy(maxNrOfRetries = 3, withinTimeRange = 5 seconds)(List(classOf[Exception]))))) val workerProps = Props(new Actor { override def postRestart(cause: Throwable) { countDownLatch.countDown() } diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala index 630da50f6d..91701596a6 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala @@ -54,7 +54,7 @@ object SupervisorSpec { var s: ActorRef = _ - override val supervisorStrategy = OneForOneStrategy(List(classOf[Exception]), maxNrOfRetries = 0) + override val supervisorStrategy = OneForOneStrategy(maxNrOfRetries = 0)(List(classOf[Exception])) def receive = { case Die ⇒ temp forward Die @@ -78,51 +78,51 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende private def child(supervisor: ActorRef, props: Props): ActorRef = Await.result((supervisor ? props).mapTo[ActorRef], timeout.duration) def temporaryActorAllForOne = { - val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), maxNrOfRetries = 0)))) + val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(maxNrOfRetries = 0)(List(classOf[Exception]))))) val temporaryActor = child(supervisor, Props(new PingPongActor(testActor))) (temporaryActor, supervisor) } def singleActorAllForOne = { - val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), - maxNrOfRetries = 3, withinTimeRange = DilatedTimeout)))) + val supervisor = system.actorOf(Props(new Supervisor( + AllForOneStrategy(maxNrOfRetries = 3, withinTimeRange = DilatedTimeout)(List(classOf[Exception]))))) val pingpong = child(supervisor, Props(new PingPongActor(testActor))) (pingpong, supervisor) } def singleActorOneForOne = { - val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), - maxNrOfRetries = 3, withinTimeRange = DilatedTimeout)))) + val supervisor = system.actorOf(Props(new Supervisor( + OneForOneStrategy(maxNrOfRetries = 3, withinTimeRange = DilatedTimeout)(List(classOf[Exception]))))) val pingpong = child(supervisor, Props(new PingPongActor(testActor))) (pingpong, supervisor) } def multipleActorsAllForOne = { - val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), - maxNrOfRetries = 3, withinTimeRange = DilatedTimeout)))) + val supervisor = system.actorOf(Props(new Supervisor( + AllForOneStrategy(maxNrOfRetries = 3, withinTimeRange = DilatedTimeout)(List(classOf[Exception]))))) val pingpong1, pingpong2, pingpong3 = child(supervisor, Props(new PingPongActor(testActor))) (pingpong1, pingpong2, pingpong3, supervisor) } def multipleActorsOneForOne = { - val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), - maxNrOfRetries = 3, withinTimeRange = DilatedTimeout)))) + val supervisor = system.actorOf(Props(new Supervisor( + OneForOneStrategy(maxNrOfRetries = 3, withinTimeRange = DilatedTimeout)(List(classOf[Exception]))))) val pingpong1, pingpong2, pingpong3 = child(supervisor, Props(new PingPongActor(testActor))) (pingpong1, pingpong2, pingpong3, supervisor) } def nestedSupervisorsAllForOne = { - val topSupervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), - maxNrOfRetries = 3, withinTimeRange = DilatedTimeout)))) + val topSupervisor = system.actorOf(Props(new Supervisor( + AllForOneStrategy(maxNrOfRetries = 3, withinTimeRange = DilatedTimeout)(List(classOf[Exception]))))) val pingpong1 = child(topSupervisor, Props(new PingPongActor(testActor))) - val middleSupervisor = child(topSupervisor, Props(new Supervisor(AllForOneStrategy(Nil, - maxNrOfRetries = 3, withinTimeRange = DilatedTimeout)))) + val middleSupervisor = child(topSupervisor, Props(new Supervisor( + AllForOneStrategy(maxNrOfRetries = 3, withinTimeRange = DilatedTimeout)(Nil)))) val pingpong2, pingpong3 = child(middleSupervisor, Props(new PingPongActor(testActor))) (pingpong1, pingpong2, pingpong3, topSupervisor) @@ -286,8 +286,8 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende "must attempt restart when exception during restart" in { val inits = new AtomicInteger(0) - val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(classOf[Exception] :: Nil, - maxNrOfRetries = 3, withinTimeRange = 10 seconds)))) + val supervisor = system.actorOf(Props(new Supervisor( + OneForOneStrategy(maxNrOfRetries = 3, withinTimeRange = 10 seconds)(classOf[Exception] :: Nil)))) val dyingProps = Props(new Actor { inits.incrementAndGet diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala index b9e3ff4a20..b84cce002c 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala @@ -23,8 +23,7 @@ class SupervisorTreeSpec extends AkkaSpec with ImplicitSender with DefaultTimeou EventFilter[ActorKilledException](occurrences = 1) intercept { within(5 seconds) { val p = Props(new Actor { - override val supervisorStrategy = OneForOneStrategy(List(classOf[Exception]), - maxNrOfRetries = 3, withinTimeRange = 1 second) + override val supervisorStrategy = OneForOneStrategy(maxNrOfRetries = 3, withinTimeRange = 1 second)(List(classOf[Exception])) def receive = { case p: Props ⇒ sender ! context.actorOf(p) } diff --git a/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala b/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala index b43ffc6d0c..285b63c2c7 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala @@ -26,7 +26,8 @@ class Ticket669Spec extends AkkaSpec with BeforeAndAfterAll with ImplicitSender "A supervised actor with lifecycle PERMANENT" should { "be able to reply on failure during preRestart" in { filterEvents(EventFilter[Exception]("test", occurrences = 1)) { - val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), 5, 10 seconds)))) + val supervisor = system.actorOf(Props(new Supervisor( + AllForOneStrategy(5, 10 seconds)(List(classOf[Exception]))))) val supervised = Await.result((supervisor ? Props[Supervised]).mapTo[ActorRef], timeout.duration) supervised.!("test")(testActor) @@ -37,8 +38,8 @@ class Ticket669Spec extends AkkaSpec with BeforeAndAfterAll with ImplicitSender "be able to reply on failure during postStop" in { filterEvents(EventFilter[Exception]("test", occurrences = 1)) { - val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), - maxNrOfRetries = 0)))) + val supervisor = system.actorOf(Props(new Supervisor( + AllForOneStrategy(maxNrOfRetries = 0)(List(classOf[Exception]))))) val supervised = Await.result((supervisor ? Props[Supervised]).mapTo[ActorRef], timeout.duration) supervised.!("test")(testActor) diff --git a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala index ced257dfc8..49b37cc506 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/TypedActorSpec.scala @@ -300,7 +300,7 @@ class TypedActorSpec extends AkkaSpec(TypedActorSpec.config) "be able to handle exceptions when calling methods" in { filterEvents(EventFilter[IllegalStateException]("expected")) { val boss = system.actorOf(Props(new Actor { - override val supervisorStrategy = OneForOneStrategy { + override val supervisorStrategy = OneForOneStrategy() { case e: IllegalStateException if e.getMessage == "expected" ⇒ SupervisorStrategy.Resume } def receive = { diff --git a/akka-actor-tests/src/test/scala/akka/event/LoggingReceiveSpec.scala b/akka-actor-tests/src/test/scala/akka/event/LoggingReceiveSpec.scala index 1857c0938b..b632bd3c40 100644 --- a/akka-actor-tests/src/test/scala/akka/event/LoggingReceiveSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/event/LoggingReceiveSpec.scala @@ -15,8 +15,8 @@ import akka.actor._ object LoggingReceiveSpec { class TestLogActor extends Actor { - override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), - maxNrOfRetries = 5, withinTimeRange = 5 seconds) + override val supervisorStrategy = + OneForOneStrategy(maxNrOfRetries = 5, withinTimeRange = 5 seconds)(List(classOf[Throwable])) def receive = { case _ ⇒ } } } diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index 86e9cb816d..7a1f640160 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -142,12 +142,12 @@ object Actor { * {{{ * class ExampleActor extends Actor { * - * override val supervisorStrategy = OneForOneStrategy({ + * override val supervisorStrategy = OneForOneStrategy(maxNrOfRetries = 10, withinTimeRange = 1 minute) { * case _: ArithmeticException ⇒ Resume * case _: NullPointerException ⇒ Restart * case _: IllegalArgumentException ⇒ Stop * case _: Exception ⇒ Escalate - * }: Decider, maxNrOfRetries = 10, withinTimeRange = 1 minute)) + * } * * def receive = { * // directly calculated reply diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index db7ca09664..2ac45e90a9 100755 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -355,7 +355,7 @@ class LocalActorRefProvider( override val supervisorStrategy = { import akka.actor.SupervisorStrategy._ - OneForOneStrategy { + OneForOneStrategy() { case _: ActorKilledException ⇒ Stop case _: ActorInitializationException ⇒ Stop case _: Exception ⇒ Restart diff --git a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala index 79166a01eb..4c7f3611a2 100644 --- a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala @@ -110,29 +110,35 @@ object SupervisorStrategy { case _: Exception ⇒ Restart case _ ⇒ Escalate } - OneForOneStrategy(defaultDecider) + OneForOneStrategy()(defaultDecider) } + /** + * Implicit conversion from `Seq` of Throwables to a `Decider`. + * This maps the given Throwables to restarts, otherwise escalates. + */ + implicit def seqThrowable2Decider(trapExit: Seq[Class[_ <: Throwable]]): Decider = makeDecider(trapExit) + type Decider = PartialFunction[Throwable, Action] type JDecider = akka.japi.Function[Throwable, Action] type CauseAction = (Class[_ <: Throwable], Action) /** - * Backwards compatible Decider builder which just checks whether one of + * Decider builder which just checks whether one of * the given Throwables matches the cause and restarts, otherwise escalates. */ def makeDecider(trapExit: Array[Class[_ <: Throwable]]): Decider = { case x ⇒ if (trapExit exists (_ isInstance x)) Restart else Escalate } /** - * Backwards compatible Decider builder which just checks whether one of + * Decider builder which just checks whether one of * the given Throwables matches the cause and restarts, otherwise escalates. */ def makeDecider(trapExit: Seq[Class[_ <: Throwable]]): Decider = { case x ⇒ if (trapExit exists (_ isInstance x)) Restart else Escalate } /** - * Backwards compatible Decider builder which just checks whether one of + * Decider builder which just checks whether one of * the given Throwables matches the cause and restarts, otherwise escalates. */ def makeDecider(trapExit: JIterable[Class[_ <: Throwable]]): Decider = makeDecider(trapExit.toSeq) @@ -209,35 +215,27 @@ abstract class SupervisorStrategy { case Escalate ⇒ false } } -} -object AllForOneStrategy { - def apply(trapExit: Seq[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Duration): AllForOneStrategy = - new AllForOneStrategy(SupervisorStrategy.makeDecider(trapExit), maxNrOfRetries, withinTimeRange) - def apply(trapExit: Seq[Class[_ <: Throwable]], maxNrOfRetries: Int): AllForOneStrategy = - apply(trapExit, maxNrOfRetries, Duration.Inf) - def apply(trapExit: Seq[Class[_ <: Throwable]], withinTimeRange: Duration): AllForOneStrategy = - apply(trapExit, -1, withinTimeRange) } /** * Restart all actors linked to the same supervisor when one fails, - * trapExit = which Throwables should be intercepted - * maxNrOfRetries = the number of times an actor is allowed to be restarted - * withinTimeRange = duration of the time window for maxNrOfRetries, Duration.Inf means no window + * @param maxNrOfRetries the number of times an actor is allowed to be restarted + * @param withinTimeRange duration of the time window for maxNrOfRetries, Duration.Inf means no window + * @param decider = mapping from Throwable to [[akka.actor.SupervisorStrategy.Action]], you can also use a + * `Seq` of Throwables which maps the given Throwables to restarts, otherwise escalates. */ -case class AllForOneStrategy(decider: SupervisorStrategy.Decider, - maxNrOfRetries: Int = -1, - withinTimeRange: Duration = Duration.Inf) extends SupervisorStrategy { +case class AllForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration = Duration.Inf)(val decider: SupervisorStrategy.Decider) + extends SupervisorStrategy { - def this(decider: SupervisorStrategy.JDecider, maxNrOfRetries: Int, withinTimeRange: Duration) = - this(SupervisorStrategy.makeDecider(decider), maxNrOfRetries, withinTimeRange) + def this(maxNrOfRetries: Int, withinTimeRange: Duration, decider: SupervisorStrategy.JDecider) = + this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(decider)) - def this(trapExit: JIterable[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Duration) = - this(SupervisorStrategy.makeDecider(trapExit), maxNrOfRetries, withinTimeRange) + def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: JIterable[Class[_ <: Throwable]]) = + this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit)) - def this(trapExit: Array[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Duration) = - this(SupervisorStrategy.makeDecider(trapExit), maxNrOfRetries, withinTimeRange) + def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: Array[Class[_ <: Throwable]]) = + this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit)) /* * this is a performance optimization to avoid re-allocating the pairs upon @@ -263,33 +261,24 @@ case class AllForOneStrategy(decider: SupervisorStrategy.Decider, } } -object OneForOneStrategy { - def apply(trapExit: Seq[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Duration): OneForOneStrategy = - new OneForOneStrategy(SupervisorStrategy.makeDecider(trapExit), maxNrOfRetries, withinTimeRange) - def apply(trapExit: Seq[Class[_ <: Throwable]], maxNrOfRetries: Int): OneForOneStrategy = - apply(trapExit, maxNrOfRetries, Duration.Inf) - def apply(trapExit: Seq[Class[_ <: Throwable]], withinTimeRange: Duration): OneForOneStrategy = - apply(trapExit, -1, withinTimeRange) -} - /** * Restart an actor when it fails - * trapExit = which Throwables should be intercepted - * maxNrOfRetries = the number of times an actor is allowed to be restarted - * withinTimeRange = duration of the time window for maxNrOfRetries, Duration.Inf means no window + * @param maxNrOfRetries the number of times an actor is allowed to be restarted + * @param withinTimeRange duration of the time window for maxNrOfRetries, Duration.Inf means no window + * @param decider = mapping from Throwable to [[akka.actor.SupervisorStrategy.Action]], you can also use a + * `Seq` of Throwables which maps the given Throwables to restarts, otherwise escalates. */ -case class OneForOneStrategy(decider: SupervisorStrategy.Decider, - maxNrOfRetries: Int = -1, - withinTimeRange: Duration = Duration.Inf) extends SupervisorStrategy { +case class OneForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration = Duration.Inf)(val decider: SupervisorStrategy.Decider) + extends SupervisorStrategy { - def this(decider: SupervisorStrategy.JDecider, maxNrOfRetries: Int, withinTimeRange: Duration) = - this(SupervisorStrategy.makeDecider(decider), maxNrOfRetries, withinTimeRange) + def this(maxNrOfRetries: Int, withinTimeRange: Duration, decider: SupervisorStrategy.JDecider) = + this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(decider)) - def this(trapExit: JIterable[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Duration) = - this(SupervisorStrategy.makeDecider(trapExit), maxNrOfRetries, withinTimeRange) + def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: JIterable[Class[_ <: Throwable]]) = + this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit)) - def this(trapExit: Array[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Duration) = - this(SupervisorStrategy.makeDecider(trapExit), maxNrOfRetries, withinTimeRange) + def this(maxNrOfRetries: Int, withinTimeRange: Duration, trapExit: Array[Class[_ <: Throwable]]) = + this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(trapExit)) /* * this is a performance optimization to avoid re-allocating the pairs upon diff --git a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala index 46afd447d6..d06896a10d 100644 --- a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala @@ -37,20 +37,21 @@ import akka.dispatch.{ MessageDispatcher, Promise } * } * } * - * private static SupervisorStrategy strategy = new OneForOneStrategy(new Function() { - * @Override - * public Action apply(Throwable t) { - * if (t instanceof ArithmeticException) { - * return resume(); - * } else if (t instanceof NullPointerException) { - * return restart(); - * } else if (t instanceof IllegalArgumentException) { - * return stop(); - * } else { - * return escalate(); + * private static SupervisorStrategy strategy = new OneForOneStrategy(10, Duration.parse("1 minute"), + * new Function() { + * @Override + * public Action apply(Throwable t) { + * if (t instanceof ArithmeticException) { + * return resume(); + * } else if (t instanceof NullPointerException) { + * return restart(); + * } else if (t instanceof IllegalArgumentException) { + * return stop(); + * } else { + * return escalate(); + * } * } - * } - * }, 10, Duration.parse("1 minute"); + * }); * * @Override * public SupervisorStrategy supervisorStrategy() { diff --git a/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java b/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java index 09be1aaaca..abf2207a1d 100644 --- a/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java +++ b/akka-docs/java/code/akka/docs/actor/FaultHandlingTestBase.java @@ -39,20 +39,21 @@ public class FaultHandlingTestBase { static public class Supervisor extends UntypedActor { //#strategy - private static SupervisorStrategy strategy = new OneForOneStrategy(new Function() { - @Override - public Action apply(Throwable t) { - if (t instanceof ArithmeticException) { - return resume(); - } else if (t instanceof NullPointerException) { - return restart(); - } else if (t instanceof IllegalArgumentException) { - return stop(); - } else { - return escalate(); - } - } - }, 10, Duration.parse("1 minute")); + private static SupervisorStrategy strategy = new OneForOneStrategy(10, Duration.parse("1 minute"), + new Function() { + @Override + public Action apply(Throwable t) { + if (t instanceof ArithmeticException) { + return resume(); + } else if (t instanceof NullPointerException) { + return restart(); + } else if (t instanceof IllegalArgumentException) { + return stop(); + } else { + return escalate(); + } + } + }); @Override public SupervisorStrategy supervisorStrategy() { @@ -76,20 +77,21 @@ public class FaultHandlingTestBase { static public class Supervisor2 extends UntypedActor { //#strategy2 - private static SupervisorStrategy strategy = new OneForOneStrategy(new Function() { - @Override - public Action apply(Throwable t) { - if (t instanceof ArithmeticException) { - return resume(); - } else if (t instanceof NullPointerException) { - return restart(); - } else if (t instanceof IllegalArgumentException) { - return stop(); - } else { - return escalate(); - } - } - }, 10, Duration.parse("1 minute")); + private static SupervisorStrategy strategy = new OneForOneStrategy(10, Duration.parse("1 minute"), + new Function() { + @Override + public Action apply(Throwable t) { + if (t instanceof ArithmeticException) { + return resume(); + } else if (t instanceof NullPointerException) { + return restart(); + } else if (t instanceof IllegalArgumentException) { + return stop(); + } else { + return escalate(); + } + } + }); @Override public SupervisorStrategy supervisorStrategy() { diff --git a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst index a8b17fa2de..d17070aeff 100644 --- a/akka-docs/project/migration-guide-1.3.x-2.0.x.rst +++ b/akka-docs/project/migration-guide-1.3.x-2.0.x.rst @@ -467,7 +467,7 @@ v1.3:: val supervisor = Supervisor( SupervisorConfig( - AllForOneStrategy(List(classOf[Exception]), 3, 1000), + OneForOneStrategy(List(classOf[Exception]), 3, 1000), Supervise( actorOf[MyActor1], Permanent) :: @@ -479,12 +479,12 @@ v1.3:: v2.0:: class MyActor extends Actor { - override val supervisorStrategy = OneForOneStrategy({ + override val supervisorStrategy = OneForOneStrategy(maxNrOfRetries = 10, withinTimeRange = 1 minute) { case _: ArithmeticException ⇒ Resume case _: NullPointerException ⇒ Restart case _: IllegalArgumentException ⇒ Stop case _: Exception ⇒ Escalate - }: Decider, maxNrOfRetries = 10, withinTimeRange = 1 minute) + } def receive = { case x => diff --git a/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala b/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala index 5e6b15078c..ca1eccb73a 100644 --- a/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSpec.scala @@ -22,12 +22,12 @@ object FaultHandlingDocSpec { import akka.actor.SupervisorStrategy._ import akka.util.duration._ - override val supervisorStrategy = OneForOneStrategy({ + override val supervisorStrategy = OneForOneStrategy(maxNrOfRetries = 10, withinTimeRange = 1 minute) { case _: ArithmeticException ⇒ Resume case _: NullPointerException ⇒ Restart case _: IllegalArgumentException ⇒ Stop case _: Exception ⇒ Escalate - }: Decider, maxNrOfRetries = 10, withinTimeRange = 1 minute) + } //#strategy def receive = { @@ -43,12 +43,12 @@ object FaultHandlingDocSpec { import akka.actor.SupervisorStrategy._ import akka.util.duration._ - override val supervisorStrategy = OneForOneStrategy({ + override val supervisorStrategy = OneForOneStrategy(maxNrOfRetries = 10, withinTimeRange = 1 minute) { case _: ArithmeticException ⇒ Resume case _: NullPointerException ⇒ Restart case _: IllegalArgumentException ⇒ Stop case _: Exception ⇒ Escalate - }: Decider, maxNrOfRetries = 10, withinTimeRange = 1 minute) + } //#strategy2 def receive = { diff --git a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala index 48a62b21eb..92476a4249 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala @@ -182,8 +182,8 @@ class TestActorRefSpec extends AkkaSpec("disp1.type=Dispatcher") with BeforeAndA override def postRestart(reason: Throwable) { counter -= 1 } }), self, "child") - override def supervisorStrategy = OneForOneStrategy(List(classOf[ActorKilledException]), - maxNrOfRetries = 5, withinTimeRange = 1 second) + override def supervisorStrategy = + OneForOneStrategy(maxNrOfRetries = 5, withinTimeRange = 1 second)(List(classOf[ActorKilledException])) def receiveT = { case "sendKill" ⇒ ref ! Kill } })) From 2ed56dea7c5aa77439bba607f5552887c544ed56 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 24 Jan 2012 10:40:24 +0100 Subject: [PATCH 123/152] Type inferer help not needed --- akka-docs/scala/fault-tolerance.rst | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/akka-docs/scala/fault-tolerance.rst b/akka-docs/scala/fault-tolerance.rst index f17cab2bb3..4c9925c947 100644 --- a/akka-docs/scala/fault-tolerance.rst +++ b/akka-docs/scala/fault-tolerance.rst @@ -31,8 +31,7 @@ that the respective limit does not apply, leaving the possibility to specify an absolute upper limit on the restarts or to make the restarts work infinitely. The match statement which forms the bulk of the body is of type ``Decider``, -which is a ``PartialFunction[Throwable, Action]``, and we need to help out the -type inferencer a bit here by ascribing that type after the closing brace. This +which is a ``PartialFunction[Throwable, Action]``. This is the piece which maps child failure types to their corresponding actions. Practical Application From d5c23bcb761d17352619dabd79e21b5140f620c4 Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 24 Jan 2012 11:33:40 +0100 Subject: [PATCH 124/152] add ExtendedActorSystem to shield ActorSystemImpl - add provider, guardian, systemGuardian and deathWatch to it - make ActorSystemImpl extend ExtendedActorSystem - use ExtendedActorSystem for creating extensions, thereby limiting the access extensions get to just those four published methods. --- .../test/java/akka/actor/JavaExtension.java | 6 +-- .../scala/akka/actor/ActorSystemSpec.scala | 4 +- .../trading/domain/TradeObserver.scala | 9 ++-- .../main/scala/akka/actor/ActorSystem.scala | 42 ++++++++++++++++++- .../src/main/scala/akka/actor/Extension.scala | 10 ++--- akka-actor/src/main/scala/akka/actor/IO.scala | 2 +- .../main/scala/akka/actor/TypedActor.scala | 4 +- .../akka/serialization/Serialization.scala | 4 +- .../SerializationExtension.scala | 4 +- .../docs/extension/ExtensionDocTestBase.java | 2 +- .../SettingsExtensionDocTestBase.java | 4 +- .../docs/extension/ExtensionDocSpec.scala | 4 +- .../extension/SettingsExtensionDocSpec.scala | 4 +- .../BeanstalkBasedMailboxExtension.scala | 2 +- .../mailbox/FileBasedMailboxExtension.scala | 2 +- .../mailbox/MongoBasedMailboxExtension.scala | 2 +- .../mailbox/RedisBasedMailboxExtension.scala | 2 +- .../ZooKeeperBasedMailboxExtension.scala | 2 +- .../testkit/CallingThreadDispatcher.scala | 26 +++++------- .../scala/akka/testkit/TestKitExtension.scala | 4 +- .../scala/akka/zeromq/ZeroMQExtension.scala | 2 +- 21 files changed, 85 insertions(+), 56 deletions(-) diff --git a/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java b/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java index 812e79e287..28b87bb5db 100644 --- a/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java +++ b/akka-actor-tests/src/test/java/akka/actor/JavaExtension.java @@ -23,15 +23,15 @@ public class JavaExtension { return TestExtensionId.TestExtensionProvider; } - public TestExtension createExtension(ActorSystemImpl i) { + public TestExtension createExtension(ExtendedActorSystem i) { return new TestExtension(i); } } static class TestExtension implements Extension { - public final ActorSystemImpl system; + public final ExtendedActorSystem system; - public TestExtension(ActorSystemImpl i) { + public TestExtension(ExtendedActorSystem i) { system = i; } } diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala index 9391ad43d1..ea8ce4c7d0 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorSystemSpec.scala @@ -15,11 +15,11 @@ class JavaExtensionSpec extends JavaExtension with JUnitSuite object TestExtension extends ExtensionId[TestExtension] with ExtensionIdProvider { def lookup = this - def createExtension(s: ActorSystemImpl) = new TestExtension(s) + def createExtension(s: ExtendedActorSystem) = new TestExtension(s) } // Dont't place inside ActorSystemSpec object, since it will not be garbage collected and reference to system remains -class TestExtension(val system: ActorSystemImpl) extends Extension +class TestExtension(val system: ExtendedActorSystem) extends Extension @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExtension$"]""") { diff --git a/akka-actor-tests/src/test/scala/akka/performance/trading/domain/TradeObserver.scala b/akka-actor-tests/src/test/scala/akka/performance/trading/domain/TradeObserver.scala index be867121f5..870a2df79d 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/trading/domain/TradeObserver.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/trading/domain/TradeObserver.scala @@ -1,11 +1,8 @@ package akka.performance.trading.domain import java.util.concurrent.atomic.AtomicInteger -import akka.actor.Extension -import akka.actor.ExtensionId -import akka.actor.ExtensionIdProvider -import akka.actor.ActorSystemImpl -import akka.actor.ActorSystem + +import akka.actor.{ ExtensionIdProvider, ExtensionId, Extension, ExtendedActorSystem, ActorSystem } abstract trait TradeObserver { def trade(bid: Bid, ask: Ask) @@ -38,5 +35,5 @@ object TotalTradeCounterExtension extends ExtensionId[TotalTradeCounter] with ExtensionIdProvider { override def lookup = TotalTradeCounterExtension - override def createExtension(system: ActorSystemImpl) = new TotalTradeCounter + override def createExtension(system: ExtendedActorSystem) = new TotalTradeCounter } \ No newline at end of file diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index b77aac491f..c6842d0dc2 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -130,6 +130,13 @@ object ActorSystem { * }}} * * Where no name is given explicitly, one will be automatically generated. + * + * Important Notice: + * + * This class is not meant to be extended by user code. If you want to + * actually roll your own Akka, it will probably be better to look into + * extending [[akka.actor.ExtendedActorSystem]] instead, but beware that you + * are completely on your own in that case! */ abstract class ActorSystem extends ActorRefFactory { import ActorSystem._ @@ -286,7 +293,40 @@ abstract class ActorSystem extends ActorRefFactory { def hasExtension(ext: ExtensionId[_ <: Extension]): Boolean } -class ActorSystemImpl(val name: String, applicationConfig: Config) extends ActorSystem { +/** + * More powerful interface to the actor system’s implementation which is presented to extensions (see [[akka.actor.Extension]]). + * + * Important Notice: + * + * This class is not meant to be extended by user code. If you want to + * actually roll your own Akka, beware that you are completely on your own in + * that case! + */ +abstract class ExtendedActorSystem extends ActorSystem { + + /** + * The ActorRefProvider is the only entity which creates all actor references within this actor system. + */ + def provider: ActorRefProvider + + /** + * The top-level supervisor of all actors created using system.actorOf(...). + */ + def guardian: InternalActorRef + + /** + * The top-level supervisor of all system-internal services like logging. + */ + def systemGuardian: InternalActorRef + + /** + * Implementation of the mechanism which is used for watch()/unwatch(). + */ + def deathWatch: DeathWatch + +} + +class ActorSystemImpl(val name: String, applicationConfig: Config) extends ExtendedActorSystem { if (!name.matches("""^\w+$""")) throw new IllegalArgumentException("invalid ActorSystem name [" + name + "], must contain only word characters (i.e. [a-zA-Z_0-9])") diff --git a/akka-actor/src/main/scala/akka/actor/Extension.scala b/akka-actor/src/main/scala/akka/actor/Extension.scala index 1850661417..f60d6afb22 100644 --- a/akka-actor/src/main/scala/akka/actor/Extension.scala +++ b/akka-actor/src/main/scala/akka/actor/Extension.scala @@ -18,10 +18,8 @@ import akka.util.ReflectiveAccess * The extension itself can be created in any way desired and has full access * to the ActorSystem implementation. * - */ - -/** - * Marker interface to signify an Akka Extension + * This trait is only a marker interface to signify an Akka Extension, see + * [[akka.actor.ExtensionKey]] for a concise way of formulating extensions. */ trait Extension @@ -47,7 +45,7 @@ trait ExtensionId[T <: Extension] { * Is used by Akka to instantiate the Extension identified by this ExtensionId, * internal use only. */ - def createExtension(system: ActorSystemImpl): T + def createExtension(system: ExtendedActorSystem): T } /** @@ -94,7 +92,7 @@ abstract class ExtensionKey[T <: Extension](implicit m: ClassManifest[T]) extend def this(clazz: Class[T]) = this()(ClassManifest.fromClass(clazz)) override def lookup(): ExtensionId[T] = this - def createExtension(system: ActorSystemImpl): T = + def createExtension(system: ExtendedActorSystem): T = ReflectiveAccess.createInstance[T](m.erasure, Array[Class[_]](classOf[ActorSystemImpl]), Array[AnyRef](system)) match { case Left(ex) ⇒ throw ex case Right(r) ⇒ r diff --git a/akka-actor/src/main/scala/akka/actor/IO.scala b/akka-actor/src/main/scala/akka/actor/IO.scala index b7942975d9..1ee38da4b9 100644 --- a/akka-actor/src/main/scala/akka/actor/IO.scala +++ b/akka-actor/src/main/scala/akka/actor/IO.scala @@ -744,7 +744,7 @@ final class IOManager private (system: ActorSystem) extends Extension { object IOManager extends ExtensionId[IOManager] with ExtensionIdProvider { override def lookup = this - override def createExtension(system: ActorSystemImpl) = new IOManager(system) + override def createExtension(system: ExtendedActorSystem) = new IOManager(system) } /** diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index 3293add151..ce487ec2ef 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -80,7 +80,7 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi override def get(system: ActorSystem): TypedActorExtension = super.get(system) def lookup() = this - def createExtension(system: ActorSystemImpl): TypedActorExtension = new TypedActorExtension(system) + def createExtension(system: ExtendedActorSystem): TypedActorExtension = new TypedActorExtension(system) /** * Returns a contextual TypedActorFactory of this extension, this means that any TypedActors created by this TypedActorExtension @@ -531,7 +531,7 @@ case class ContextualTypedActorFactory(typedActor: TypedActorExtension, actorFac override def isTypedActor(proxyOrNot: AnyRef): Boolean = typedActor.isTypedActor(proxyOrNot) } -class TypedActorExtension(system: ActorSystemImpl) extends TypedActorFactory with Extension { +class TypedActorExtension(system: ExtendedActorSystem) extends TypedActorFactory with Extension { import TypedActor._ //Import the goodies from the companion object protected def actorFactory: ActorRefFactory = system protected def typedActor = this diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index eaa0ce0925..78cb370b68 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -9,7 +9,7 @@ import akka.util.ReflectiveAccess import scala.util.DynamicVariable import com.typesafe.config.Config import akka.config.ConfigurationException -import akka.actor.{ Extension, ActorSystem, ActorSystemImpl } +import akka.actor.{ Extension, ActorSystem, ExtendedActorSystem } case class NoSerializerFoundException(m: String) extends AkkaException(m) @@ -55,7 +55,7 @@ object Serialization { * Serialization module. Contains methods for serialization and deserialization as well as * locating a Serializer for a particular class as defined in the mapping in the 'akka.conf' file. */ -class Serialization(val system: ActorSystemImpl) extends Extension { +class Serialization(val system: ExtendedActorSystem) extends Extension { import Serialization._ val settings = new Settings(system.settings.config) diff --git a/akka-actor/src/main/scala/akka/serialization/SerializationExtension.scala b/akka-actor/src/main/scala/akka/serialization/SerializationExtension.scala index 9d9815f412..f96aa26e0c 100644 --- a/akka-actor/src/main/scala/akka/serialization/SerializationExtension.scala +++ b/akka-actor/src/main/scala/akka/serialization/SerializationExtension.scala @@ -3,7 +3,7 @@ */ package akka.serialization -import akka.actor.{ ActorSystem, ExtensionId, ExtensionIdProvider, ActorSystemImpl } +import akka.actor.{ ActorSystem, ExtensionId, ExtensionIdProvider, ExtendedActorSystem } /** * SerializationExtension is an Akka Extension to interact with the Serialization @@ -12,5 +12,5 @@ import akka.actor.{ ActorSystem, ExtensionId, ExtensionIdProvider, ActorSystemIm object SerializationExtension extends ExtensionId[Serialization] with ExtensionIdProvider { override def get(system: ActorSystem): Serialization = super.get(system) override def lookup = SerializationExtension - override def createExtension(system: ActorSystemImpl): Serialization = new Serialization(system) + override def createExtension(system: ExtendedActorSystem): Serialization = new Serialization(system) } \ No newline at end of file diff --git a/akka-docs/java/code/akka/docs/extension/ExtensionDocTestBase.java b/akka-docs/java/code/akka/docs/extension/ExtensionDocTestBase.java index 6d62eb5bb8..11dfe4c198 100644 --- a/akka-docs/java/code/akka/docs/extension/ExtensionDocTestBase.java +++ b/akka-docs/java/code/akka/docs/extension/ExtensionDocTestBase.java @@ -42,7 +42,7 @@ public class ExtensionDocTestBase { //This method will be called by Akka // to instantiate our Extension - public CountExtensionImpl createExtension(ActorSystemImpl system) { + public CountExtensionImpl createExtension(ExtendedActorSystem system) { return new CountExtensionImpl(); } } diff --git a/akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTestBase.java b/akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTestBase.java index 6ca1c371d9..1bf6ce7c36 100644 --- a/akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTestBase.java +++ b/akka-docs/java/code/akka/docs/extension/SettingsExtensionDocTestBase.java @@ -8,7 +8,7 @@ import akka.actor.Extension; import akka.actor.AbstractExtensionId; import akka.actor.ExtensionIdProvider; import akka.actor.ActorSystem; -import akka.actor.ActorSystemImpl; +import akka.actor.ExtendedActorSystem; import akka.util.Duration; import com.typesafe.config.Config; import java.util.concurrent.TimeUnit; @@ -44,7 +44,7 @@ public class SettingsExtensionDocTestBase { return Settings.SettingsProvider; } - public SettingsImpl createExtension(ActorSystemImpl system) { + public SettingsImpl createExtension(ExtendedActorSystem system) { return new SettingsImpl(system.settings().config()); } } diff --git a/akka-docs/scala/code/akka/docs/extension/ExtensionDocSpec.scala b/akka-docs/scala/code/akka/docs/extension/ExtensionDocSpec.scala index 1f4d777ad9..0c778a4812 100644 --- a/akka-docs/scala/code/akka/docs/extension/ExtensionDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/extension/ExtensionDocSpec.scala @@ -23,7 +23,7 @@ class CountExtensionImpl extends Extension { //#extensionid import akka.actor.ExtensionId import akka.actor.ExtensionIdProvider -import akka.actor.ActorSystemImpl +import akka.actor.ExtendedActorSystem object CountExtension extends ExtensionId[CountExtensionImpl] @@ -36,7 +36,7 @@ object CountExtension //This method will be called by Akka // to instantiate our Extension - override def createExtension(system: ActorSystemImpl) = new CountExtensionImpl + override def createExtension(system: ExtendedActorSystem) = new CountExtensionImpl } //#extensionid diff --git a/akka-docs/scala/code/akka/docs/extension/SettingsExtensionDocSpec.scala b/akka-docs/scala/code/akka/docs/extension/SettingsExtensionDocSpec.scala index ed5ec66517..05765d27a5 100644 --- a/akka-docs/scala/code/akka/docs/extension/SettingsExtensionDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/extension/SettingsExtensionDocSpec.scala @@ -7,7 +7,7 @@ package akka.docs.extension import akka.actor.Extension import akka.actor.ExtensionId import akka.actor.ExtensionIdProvider -import akka.actor.ActorSystemImpl +import akka.actor.ExtendedActorSystem import akka.util.Duration import com.typesafe.config.Config import java.util.concurrent.TimeUnit @@ -29,7 +29,7 @@ object Settings extends ExtensionId[SettingsImpl] with ExtensionIdProvider { override def lookup = Settings - override def createExtension(system: ActorSystemImpl) = new SettingsImpl(system.settings.config) + override def createExtension(system: ExtendedActorSystem) = new SettingsImpl(system.settings.config) } //#extensionid diff --git a/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailboxExtension.scala b/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailboxExtension.scala index 91e8085778..36ab10393a 100644 --- a/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailboxExtension.scala +++ b/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailboxExtension.scala @@ -11,7 +11,7 @@ import akka.actor._ object BeanstalkBasedMailboxExtension extends ExtensionId[BeanstalkMailboxSettings] with ExtensionIdProvider { override def get(system: ActorSystem): BeanstalkMailboxSettings = super.get(system) def lookup() = this - def createExtension(system: ActorSystemImpl) = new BeanstalkMailboxSettings(system.settings.config) + def createExtension(system: ExtendedActorSystem) = new BeanstalkMailboxSettings(system.settings.config) } class BeanstalkMailboxSettings(val config: Config) extends Extension { diff --git a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FileBasedMailboxExtension.scala b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FileBasedMailboxExtension.scala index c09bfc9cb3..f7e6527499 100644 --- a/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FileBasedMailboxExtension.scala +++ b/akka-durable-mailboxes/akka-file-mailbox/src/main/scala/akka/actor/mailbox/FileBasedMailboxExtension.scala @@ -11,7 +11,7 @@ import akka.actor._ object FileBasedMailboxExtension extends ExtensionId[FileBasedMailboxSettings] with ExtensionIdProvider { override def get(system: ActorSystem): FileBasedMailboxSettings = super.get(system) def lookup() = this - def createExtension(system: ActorSystemImpl) = new FileBasedMailboxSettings(system.settings.config) + def createExtension(system: ExtendedActorSystem) = new FileBasedMailboxSettings(system.settings.config) } class FileBasedMailboxSettings(val config: Config) extends Extension { diff --git a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailboxExtension.scala b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailboxExtension.scala index 0176fc09f3..fac0ad9050 100644 --- a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailboxExtension.scala +++ b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailboxExtension.scala @@ -11,7 +11,7 @@ import akka.actor._ object MongoBasedMailboxExtension extends ExtensionId[MongoBasedMailboxSettings] with ExtensionIdProvider { override def get(system: ActorSystem): MongoBasedMailboxSettings = super.get(system) def lookup() = this - def createExtension(system: ActorSystemImpl) = new MongoBasedMailboxSettings(system.settings.config) + def createExtension(system: ExtendedActorSystem) = new MongoBasedMailboxSettings(system.settings.config) } class MongoBasedMailboxSettings(val config: Config) extends Extension { diff --git a/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailboxExtension.scala b/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailboxExtension.scala index 17ce479244..629f08b145 100644 --- a/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailboxExtension.scala +++ b/akka-durable-mailboxes/akka-redis-mailbox/src/main/scala/akka/actor/mailbox/RedisBasedMailboxExtension.scala @@ -9,7 +9,7 @@ import akka.actor._ object RedisBasedMailboxExtension extends ExtensionId[RedisBasedMailboxSettings] with ExtensionIdProvider { override def get(system: ActorSystem): RedisBasedMailboxSettings = super.get(system) def lookup() = this - def createExtension(system: ActorSystemImpl) = new RedisBasedMailboxSettings(system.settings.config) + def createExtension(system: ExtendedActorSystem) = new RedisBasedMailboxSettings(system.settings.config) } class RedisBasedMailboxSettings(val config: Config) extends Extension { diff --git a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailboxExtension.scala b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailboxExtension.scala index de2f2d586f..4f3dcfb42f 100644 --- a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailboxExtension.scala +++ b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailboxExtension.scala @@ -11,7 +11,7 @@ import akka.actor._ object ZooKeeperBasedMailboxExtension extends ExtensionId[ZooKeeperBasedMailboxSettings] with ExtensionIdProvider { override def get(system: ActorSystem): ZooKeeperBasedMailboxSettings = super.get(system) def lookup() = this - def createExtension(system: ActorSystemImpl) = new ZooKeeperBasedMailboxSettings(system.settings.config) + def createExtension(system: ExtendedActorSystem) = new ZooKeeperBasedMailboxSettings(system.settings.config) } class ZooKeeperBasedMailboxSettings(val config: Config) extends Extension { diff --git a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala index a3a1982e9f..96d9f8241a 100644 --- a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala +++ b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala @@ -3,26 +3,20 @@ */ package akka.testkit -import akka.event.Logging.{ Warning, Error } +import java.lang.ref.WeakReference import java.util.concurrent.locks.ReentrantLock import java.util.LinkedList -import java.util.concurrent.RejectedExecutionException -import akka.util.Switch -import java.lang.ref.WeakReference + import scala.annotation.tailrec -import akka.actor.{ ActorCell, ActorRef, ActorSystem } -import akka.dispatch._ -import akka.actor.Scheduler -import akka.event.EventStream -import akka.util.Duration -import akka.util.duration._ -import java.util.concurrent.TimeUnit -import akka.actor.ExtensionId -import akka.actor.ExtensionIdProvider -import akka.actor.ActorSystemImpl -import akka.actor.Extension + import com.typesafe.config.Config +import CallingThreadDispatcher.Id +import akka.actor.{ ExtensionIdProvider, ExtensionId, Extension, ExtendedActorSystem, ActorRef, ActorCell } +import akka.dispatch.{ TaskInvocation, SystemMessage, Suspend, Resume, MessageDispatcherConfigurator, MessageDispatcher, Mailbox, Envelope, DispatcherPrerequisites, DefaultSystemMessageQueue } +import akka.util.duration.intToDurationInt +import akka.util.{ Switch, Duration } + /* * Locking rules: * @@ -42,7 +36,7 @@ import com.typesafe.config.Config private[testkit] object CallingThreadDispatcherQueues extends ExtensionId[CallingThreadDispatcherQueues] with ExtensionIdProvider { override def lookup = CallingThreadDispatcherQueues - override def createExtension(system: ActorSystemImpl): CallingThreadDispatcherQueues = new CallingThreadDispatcherQueues + override def createExtension(system: ExtendedActorSystem): CallingThreadDispatcherQueues = new CallingThreadDispatcherQueues } private[testkit] class CallingThreadDispatcherQueues extends Extension { diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKitExtension.scala b/akka-testkit/src/main/scala/akka/testkit/TestKitExtension.scala index 89c40f48f4..ada5a4fd30 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestKitExtension.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestKitExtension.scala @@ -6,11 +6,11 @@ package akka.testkit import com.typesafe.config.Config import akka.util.Duration import java.util.concurrent.TimeUnit.MILLISECONDS -import akka.actor.{ ExtensionId, ActorSystem, Extension, ActorSystemImpl } +import akka.actor.{ ExtensionId, ActorSystem, Extension, ExtendedActorSystem } object TestKitExtension extends ExtensionId[TestKitSettings] { override def get(system: ActorSystem): TestKitSettings = super.get(system) - def createExtension(system: ActorSystemImpl): TestKitSettings = new TestKitSettings(system.settings.config) + def createExtension(system: ExtendedActorSystem): TestKitSettings = new TestKitSettings(system.settings.config) } class TestKitSettings(val config: Config) extends Extension { diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala index f0dee326b2..c0486fa3ab 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala @@ -24,7 +24,7 @@ case class ZeroMQVersion(major: Int, minor: Int, patch: Int) { */ object ZeroMQExtension extends ExtensionId[ZeroMQExtension] with ExtensionIdProvider { def lookup() = this - def createExtension(system: ActorSystemImpl) = new ZeroMQExtension(system) + def createExtension(system: ExtendedActorSystem) = new ZeroMQExtension(system) private val minVersionString = "2.1.0" private val minVersion = JZMQ.makeVersion(2, 1, 0) From 36ef8820b8fa25f8d3756b1b618840179d09b4de Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 24 Jan 2012 12:13:13 +0100 Subject: [PATCH 125/152] Adding docs and samples for onSuccess, onFailure, onComplete, filter, or, zip and successful/failed --- .../code/akka/docs/future/FutureDocSpec.scala | 76 +++++++++++++++++++ akka-docs/scala/futures.rst | 57 +++++++++++--- 2 files changed, 123 insertions(+), 10 deletions(-) diff --git a/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala b/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala index 69bfe31899..d1e9532548 100644 --- a/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala @@ -125,12 +125,22 @@ class FutureDocSpec extends AkkaSpec { //#flat-map } + "demonstrate usage of filter" in { + //#filter + val future1 = Promise.successful(4) + val future2 = future1.filter(_ % 2 == 0) + val result = Await.result(future2, 1 second) + result must be(4) + //#filter + } + "demonstrate usage of for comprehension" in { //#for-comprehension val f = for { a ← Future(10 / 2) // 10 / 2 = 5 b ← Future(a + 1) // 5 + 1 = 6 c ← Future(a - 1) // 5 - 1 = 4 + if c > 3 // Future.filter } yield b * c // 6 * 4 = 24 // Note that the execution of futures a, b, and c @@ -250,4 +260,70 @@ class FutureDocSpec extends AkkaSpec { Await.result(future, 1 second) must be(0) } + "demonstrate usage of zip" in { + val future1 = Future { "foo" } + val future2 = Future { "bar" } + //#zip + val future3 = future1 zip future2 map { case (a, b) ⇒ a + " " + b } + //#zip + Await.result(future3, 1 second) must be("foo bar") + } + + "demonstrate usage of or" in { + val future1 = Future { "foo" } + val future2 = Future { "bar" } + val future3 = Future { "pigdog" } + //#or + val future4 = future1 or future2 or future3 + //#or + Await.result(future4, 1 second) must be("foo") + } + + "demonstrate usage of onSuccess & onFailure & onComplete" in { + { + val future = Future { "foo" } + //#onSuccess + future onSuccess { + case "bar" ⇒ println("Got my bar alright!") + case x: String ⇒ println("Got some random string: " + x) + } + //#onSuccess + Await.result(future, 1 second) must be("foo") + } + { + val future = Promise.failed[String](new IllegalStateException("OHNOES")) + //#onFailure + future onFailure { + case ise: IllegalStateException if ise.getMessage == "OHNOES" ⇒ + //OHNOES! We are in deep trouble, do something! + case e: Exception ⇒ + //Do something else + } + //#onFailure + } + { + val future = Future { "foo" } + def doSomethingOnSuccess(r: String) = () + def doSomethingOnFailure(t: Throwable) = () + //#onComplete + future onComplete { + case Right(result) ⇒ doSomethingOnSuccess(result) //Right == Success + case Left(failure) ⇒ doSomethingOnFailure(failure) //Left == Failure + } + //#onComplete + Await.result(future, 1 second) must be("foo") + } + } + + "demonstrate usage of Promise.success & Promise.failed" in { + //#successful + val future = Promise.successful("Yay!") + //#successful + //#failed + val otherFuture = Promise.failed[String](new IllegalArgumentException("Bang!")) + //#failed + Await.result(future, 1 second) must be("Yay!") + intercept[IllegalArgumentException] { Await.result(otherFuture, 1 second) } + } + } diff --git a/akka-docs/scala/futures.rst b/akka-docs/scala/futures.rst index 5707fcf029..558995de89 100644 --- a/akka-docs/scala/futures.rst +++ b/akka-docs/scala/futures.rst @@ -42,6 +42,16 @@ A common use case within Akka is to have some computation performed concurrently In the above code the block passed to ``Future`` will be executed by the default ``Dispatcher``, with the return value of the block used to complete the ``Future`` (in this case, the result would be the string: "HelloWorld"). Unlike a ``Future`` that is returned from an ``Actor``, this ``Future`` is properly typed, and we also avoid the overhead of managing an ``Actor``. +You can also create already completed Futures using the ``Promise`` companion, which can be either successes: + +.. includecode:: code/akka/docs/future/FutureDocSpec.scala + :include: successful + +Or failures: + +.. includecode:: code/akka/docs/future/FutureDocSpec.scala + :include: failed + Functional Futures ------------------ @@ -67,10 +77,15 @@ The ``map`` method is fine if we are modifying a single ``Future``, but if 2 or .. includecode:: code/akka/docs/future/FutureDocSpec.scala :include: flat-map +If you need to do conditional propagation, you can use ``filter``: + +.. includecode:: code/akka/docs/future/FutureDocSpec.scala + :include: filter + For Comprehensions ^^^^^^^^^^^^^^^^^^ -Since ``Future`` has a ``map`` and ``flatMap`` method it can be easily used in a 'for comprehension': +Since ``Future`` has a ``map``, ``filter` and ``flatMap`` method it can be easily used in a 'for comprehension': .. includecode:: code/akka/docs/future/FutureDocSpec.scala :include: for-comprehension @@ -111,7 +126,9 @@ This is the same result as this example: But it may be faster to use ``traverse`` as it doesn't have to create an intermediate ``List[Future[Int]]``. -Then there's a method that's called ``fold`` that takes a start-value, a sequence of ``Future``\s and a function from the type of the start-value and the type of the futures and returns something with the same type as the start-value, and then applies the function to all elements in the sequence of futures, non-blockingly, the execution will run on the Thread of the last completing Future in the sequence. +Then there's a method that's called ``fold`` that takes a start-value, a sequence of ``Future``\s and a function +from the type of the start-value and the type of the futures and returns something with the same type as the start-value, +and then applies the function to all elements in the sequence of futures, non-blockingly, the execution will run on the Thread of the last completing Future in the sequence. .. includecode:: code/akka/docs/future/FutureDocSpec.scala :include: fold @@ -119,24 +136,44 @@ Then there's a method that's called ``fold`` that takes a start-value, a sequenc That's all it takes! -If the sequence passed to ``fold`` is empty, it will return the start-value, in the case above, that will be 0. In some cases you don't have a start-value and you're able to use the value of the first completing Future in the sequence as the start-value, you can use ``reduce``, it works like this: +If the sequence passed to ``fold`` is empty, it will return the start-value, in the case above, that will be 0. +In some cases you don't have a start-value and you're able to use the value of the first completing Future in the sequence as the start-value, you can use ``reduce``, it works like this: .. includecode:: code/akka/docs/future/FutureDocSpec.scala :include: reduce -Same as with ``fold``, the execution will be done by the Thread that completes the last of the Futures, you can also parallelize it by chunking your futures into sub-sequences and reduce them, and then reduce the reduced results again. +Same as with ``fold``, the execution will be done by the Thread that completes the last of the Futures, ` +you can also parallelize it by chunking your futures into sub-sequences and reduce them, and then reduce the reduced results again. -This is just a sample of what can be done, but to use more advanced techniques it is easier to take advantage of Scalaz. +Callbacks +--------- +Sometimes you just want to listen to a ``Future`` being completed, and react to that not by creating a new Future, but by side-effecting. +For this Akka supports ``onComplete``, ``onSuccess`` and ``onFailure``, of which the latter two are specializations of the first. -Scalaz -^^^^^^ +.. includecode:: code/akka/docs/future/FutureDocSpec.scala + :include: onSuccess -There is also an `Akka-Scalaz`_ project created by Derek Williams for a more -complete support of programming in a functional style. +.. includecode:: code/akka/docs/future/FutureDocSpec.scala + :include: onFailure -.. _Akka-Scalaz: https://github.com/derekjw/akka-scalaz +.. includecode:: code/akka/docs/future/FutureDocSpec.scala + :include: onComplete +Auxiliary methods +----------------- + +``Future`` ``or`` combines 2 Futures into a new ``Future``, and will hold the successful value of the second ``Future`` +if the first ``Future`` fails. + +.. includecode:: code/akka/docs/future/FutureDocSpec.scala + :include: or + +You can also combine two Futures into a new ``Future`` that will hold a tuple of the two Futures successful results, +using the ``zip`` operation. + +.. includecode:: code/akka/docs/future/FutureDocSpec.scala + :include: zip Exceptions ---------- From f4ee0969b072063a5bf7247cc2fd49726dcfea85 Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 24 Jan 2012 12:14:05 +0100 Subject: [PATCH 126/152] add implicit Decider conv. for Iterable[CauseAction] --- .../src/main/scala/akka/actor/FaultHandling.scala | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala index 4c7f3611a2..8d26567c37 100644 --- a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala @@ -45,7 +45,16 @@ case class ChildRestartStats(val child: ActorRef, var maxNrOfRetriesCount: Int = } } -object SupervisorStrategy { +trait SupervisorStrategyLowPriorityImplicits { this: SupervisorStrategy.type ⇒ + + /** + * Implicit conversion from `Seq` of Cause-Action pairs to a `Decider`. See makeDecider(causeAction). + */ + implicit def seqCauseAction2Decider(trapExit: Iterable[CauseAction]): Decider = makeDecider(trapExit) + // the above would clash with seqThrowable2Decider for empty lists +} + +object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits { sealed trait Action /** From c351aef39cc68fdc8cd481b58f34d60deddfca1d Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 24 Jan 2012 13:59:23 +0100 Subject: [PATCH 127/152] Correcting spelling errors and fixing formatting of futures.rst --- akka-docs/scala/futures.rst | 95 +++++++++++++++++++++++++++---------- 1 file changed, 70 insertions(+), 25 deletions(-) diff --git a/akka-docs/scala/futures.rst b/akka-docs/scala/futures.rst index 558995de89..5d0baa5ca7 100644 --- a/akka-docs/scala/futures.rst +++ b/akka-docs/scala/futures.rst @@ -10,37 +10,48 @@ Futures (Scala) Introduction ------------ -In Akka, a `Future `_ is a data structure used to retrieve the result of some concurrent operation. This operation is usually performed by an ``Actor`` or by the ``Dispatcher`` directly. This result can be accessed synchronously (blocking) or asynchronously (non-blocking). +In Akka, a `Future `_ is a data structure used to +retrieve the result of some concurrent operation. This operation is usually performed by an ``Actor`` +or by the ``Dispatcher`` directly. This result can be accessed synchronously (blocking) or asynchronously (non-blocking). Use with Actors --------------- -There are generally two ways of getting a reply from an ``Actor``: the first is by a sent message (``actor ! msg``), which only works if the original sender was an ``Actor``) and the second is through a ``Future``. +There are generally two ways of getting a reply from an ``Actor``: the first is by a sent message (``actor ! msg``), +which only works if the original sender was an ``Actor``) and the second is through a ``Future``. Using an ``Actor``\'s ``?`` method to send a message will return a Future. To wait for and retrieve the actual result the simplest method is: .. includecode:: code/akka/docs/future/FutureDocSpec.scala :include: ask-blocking -This will cause the current thread to block and wait for the ``Actor`` to 'complete' the ``Future`` with it's reply. Blocking is discouraged though as it can cause performance problem. -The blocking operations are located in ``Await.result`` and ``Await.ready`` to make it easy to spot where blocking occurs. Alternatives to blocking are discussed further within this documentation. -Also note that the ``Future`` returned by an ``Actor`` is a ``Future[Any]`` since an ``Actor`` is dynamic. That is why the ``asInstanceOf`` is used in the above sample. +This will cause the current thread to block and wait for the ``Actor`` to 'complete' the ``Future`` with it's reply. +Blocking is discouraged though as it will cause performance problems. +The blocking operations are located in ``Await.result`` and ``Await.ready`` to make it easy to spot where blocking occurs. +Alternatives to blocking are discussed further within this documentation. Also note that the ``Future`` returned by +an ``Actor`` is a ``Future[Any]`` since an ``Actor`` is dynamic. That is why the ``asInstanceOf`` is used in the above sample. When using non-blocking it is better to use the ``mapTo`` method to safely try to cast a ``Future`` to an expected type: .. includecode:: code/akka/docs/future/FutureDocSpec.scala :include: map-to -The ``mapTo`` method will return a new ``Future`` that contains the result if the cast was successful, or a ``ClassCastException`` if not. Handling ``Exception``\s will be discussed further within this documentation. +The ``mapTo`` method will return a new ``Future`` that contains the result if the cast was successful, +or a ``ClassCastException`` if not. Handling ``Exception``\s will be discussed further within this documentation. Use Directly ------------ -A common use case within Akka is to have some computation performed concurrently without needing the extra utility of an ``Actor``. If you find yourself creating a pool of ``Actor``\s for the sole reason of performing a calculation in parallel, there is an easier (and faster) way: +A common use case within Akka is to have some computation performed concurrently without needing the extra utility of an ``Actor``. +If you find yourself creating a pool of ``Actor``\s for the sole reason of performing a calculation in parallel, +there is an easier (and faster) way: .. includecode:: code/akka/docs/future/FutureDocSpec.scala :include: future-eval -In the above code the block passed to ``Future`` will be executed by the default ``Dispatcher``, with the return value of the block used to complete the ``Future`` (in this case, the result would be the string: "HelloWorld"). Unlike a ``Future`` that is returned from an ``Actor``, this ``Future`` is properly typed, and we also avoid the overhead of managing an ``Actor``. +In the above code the block passed to ``Future`` will be executed by the default ``Dispatcher``, +with the return value of the block used to complete the ``Future`` (in this case, the result would be the string: "HelloWorld"). +Unlike a ``Future`` that is returned from an ``Actor``, this ``Future`` is properly typed, +and we also avoid the overhead of managing an ``Actor``. You can also create already completed Futures using the ``Promise`` companion, which can be either successes: @@ -55,19 +66,28 @@ Or failures: Functional Futures ------------------ -Akka's ``Future`` has several monadic methods that are very similar to the ones used by Scala's collections. These allow you to create 'pipelines' or 'streams' that the result will travel through. +Akka's ``Future`` has several monadic methods that are very similar to the ones used by Scala's collections. +These allow you to create 'pipelines' or 'streams' that the result will travel through. Future is a Monad ^^^^^^^^^^^^^^^^^ -The first method for working with ``Future`` functionally is ``map``. This method takes a ``Function`` which performs some operation on the result of the ``Future``, and returning a new result. The return value of the ``map`` method is another ``Future`` that will contain the new result: +The first method for working with ``Future`` functionally is ``map``. This method takes a ``Function`` +which performs some operation on the result of the ``Future``, and returning a new result. +The return value of the ``map`` method is another ``Future`` that will contain the new result: .. includecode:: code/akka/docs/future/FutureDocSpec.scala :include: map -In this example we are joining two strings together within a ``Future``. Instead of waiting for this to complete, we apply our function that calculates the length of the string using the ``map`` method. Now we have a second ``Future`` that will eventually contain an ``Int``. When our original ``Future`` completes, it will also apply our function and complete the second ``Future`` with it's result. When we finally get the result, it will contain the number 10. Our original ``Future`` still contains the string "HelloWorld" and is unaffected by the ``map``. +In this example we are joining two strings together within a ``Future``. Instead of waiting for this to complete, +we apply our function that calculates the length of the string using the ``map`` method. +Now we have a second ``Future`` that will eventually contain an ``Int``. +When our original ``Future`` completes, it will also apply our function and complete the second ``Future`` with its result. +When we finally get the result, it will contain the number 10. Our original ``Future`` still contains the +string "HelloWorld" and is unaffected by the ``map``. -The ``map`` method is fine if we are modifying a single ``Future``, but if 2 or more ``Future``\s are involved ``map`` will not allow you to combine them together: +The ``map`` method is fine if we are modifying a single ``Future``, +but if 2 or more ``Future``\s are involved ``map`` will not allow you to combine them together: .. includecode:: code/akka/docs/future/FutureDocSpec.scala :include: wrong-nested-map @@ -90,31 +110,47 @@ Since ``Future`` has a ``map``, ``filter` and ``flatMap`` method it can be easil .. includecode:: code/akka/docs/future/FutureDocSpec.scala :include: for-comprehension -Something to keep in mind when doing this is even though it looks like parts of the above example can run in parallel, each step of the for comprehension is run sequentially. This will happen on separate threads for each step but there isn't much benefit over running the calculations all within a single ``Future``. The real benefit comes when the ``Future``\s are created first, and then combining them together. +Something to keep in mind when doing this is even though it looks like parts of the above example can run in parallel, +each step of the for comprehension is run sequentially. This will happen on separate threads for each step but +there isn't much benefit over running the calculations all within a single ``Future``. +The real benefit comes when the ``Future``\s are created first, and then combining them together. Composing Futures ^^^^^^^^^^^^^^^^^ -The example for comprehension above is an example of composing ``Future``\s. A common use case for this is combining the replies of several ``Actor``\s into a single calculation without resorting to calling ``Await.result`` or ``Await.ready`` to block for each result. First an example of using ``Await.result``: +The example for comprehension above is an example of composing ``Future``\s. +A common use case for this is combining the replies of several ``Actor``\s into a single calculation +without resorting to calling ``Await.result`` or ``Await.ready`` to block for each result. +First an example of using ``Await.result``: .. includecode:: code/akka/docs/future/FutureDocSpec.scala :include: composing-wrong -Here we wait for the results from the first 2 ``Actor``\s before sending that result to the third ``Actor``. We called ``Await.result`` 3 times, which caused our little program to block 3 times before getting our final result. Now compare that to this example: +Here we wait for the results from the first 2 ``Actor``\s before sending that result to the third ``Actor``. +We called ``Await.result`` 3 times, which caused our little program to block 3 times before getting our final result. +Now compare that to this example: .. includecode:: code/akka/docs/future/FutureDocSpec.scala :include: composing -Here we have 2 actors processing a single message each. Once the 2 results are available (note that we don't block to get these results!), they are being added together and sent to a third ``Actor``, which replies with a string, which we assign to 'result'. +Here we have 2 actors processing a single message each. Once the 2 results are available +(note that we don't block to get these results!), they are being added together and sent to a third ``Actor``, +which replies with a string, which we assign to 'result'. -This is fine when dealing with a known amount of Actors, but can grow unwieldy if we have more then a handful. The ``sequence`` and ``traverse`` helper methods can make it easier to handle more complex use cases. Both of these methods are ways of turning, for a subclass ``T`` of ``Traversable``, ``T[Future[A]]`` into a ``Future[T[A]]``. For example: +This is fine when dealing with a known amount of Actors, but can grow unwieldy if we have more then a handful. +The ``sequence`` and ``traverse`` helper methods can make it easier to handle more complex use cases. +Both of these methods are ways of turning, for a subclass ``T`` of ``Traversable``, ``T[Future[A]]`` into a ``Future[T[A]]``. +For example: .. includecode:: code/akka/docs/future/FutureDocSpec.scala :include: sequence-ask -To better explain what happened in the example, ``Future.sequence`` is taking the ``List[Future[Int]]`` and turning it into a ``Future[List[Int]]``. We can then use ``map`` to work with the ``List[Int]`` directly, and we find the sum of the ``List``. +To better explain what happened in the example, ``Future.sequence`` is taking the ``List[Future[Int]]`` +and turning it into a ``Future[List[Int]]``. We can then use ``map`` to work with the ``List[Int]`` directly, a +nd we find the sum of the ``List``. -The ``traverse`` method is similar to ``sequence``, but it takes a ``T[A]`` and a function ``A => Future[B]`` to return a ``Future[T[B]]``, where ``T`` is again a subclass of Traversable. For example, to use ``traverse`` to sum the first 100 odd numbers: +The ``traverse`` method is similar to ``sequence``, but it takes a ``T[A]`` and a function ``A => Future[B]`` to return a ``Future[T[B]]``, +where ``T`` is again a subclass of Traversable. For example, to use ``traverse`` to sum the first 100 odd numbers: .. includecode:: code/akka/docs/future/FutureDocSpec.scala :include: traverse @@ -128,7 +164,8 @@ But it may be faster to use ``traverse`` as it doesn't have to create an interme Then there's a method that's called ``fold`` that takes a start-value, a sequence of ``Future``\s and a function from the type of the start-value and the type of the futures and returns something with the same type as the start-value, -and then applies the function to all elements in the sequence of futures, non-blockingly, the execution will run on the Thread of the last completing Future in the sequence. +and then applies the function to all elements in the sequence of futures, non-blockingly, +the execution will run on the Thread of the last completing Future in the sequence. .. includecode:: code/akka/docs/future/FutureDocSpec.scala :include: fold @@ -137,7 +174,8 @@ That's all it takes! If the sequence passed to ``fold`` is empty, it will return the start-value, in the case above, that will be 0. -In some cases you don't have a start-value and you're able to use the value of the first completing Future in the sequence as the start-value, you can use ``reduce``, it works like this: +In some cases you don't have a start-value and you're able to use the value of the first completing Future in the sequence +as the start-value, you can use ``reduce``, it works like this: .. includecode:: code/akka/docs/future/FutureDocSpec.scala :include: reduce @@ -163,7 +201,7 @@ For this Akka supports ``onComplete``, ``onSuccess`` and ``onFailure``, of which Auxiliary methods ----------------- -``Future`` ``or`` combines 2 Futures into a new ``Future``, and will hold the successful value of the second ``Future`` +``Future`` ``or`` combines 2 Futures into a new ``Future``, and will hold the successful value of the second ``Future` if the first ``Future`` fails. .. includecode:: code/akka/docs/future/FutureDocSpec.scala @@ -178,12 +216,19 @@ using the ``zip`` operation. Exceptions ---------- -Since the result of a ``Future`` is created concurrently to the rest of the program, exceptions must be handled differently. It doesn't matter if an ``Actor`` or the dispatcher is completing the ``Future``, if an ``Exception`` is caught the ``Future`` will contain it instead of a valid result. If a ``Future`` does contain an ``Exception``, calling ``Await.result`` will cause it to be thrown again so it can be handled properly. +Since the result of a ``Future`` is created concurrently to the rest of the program, exceptions must be handled differently. +It doesn't matter if an ``Actor`` or the dispatcher is completing the ``Future``, +if an ``Exception`` is caught the ``Future`` will contain it instead of a valid result. +If a ``Future`` does contain an ``Exception``, calling ``Await.result`` will cause it to be thrown again so it can be handled properly. -It is also possible to handle an ``Exception`` by returning a different result. This is done with the ``recover`` method. For example: +It is also possible to handle an ``Exception`` by returning a different result. +This is done with the ``recover`` method. For example: .. includecode:: code/akka/docs/future/FutureDocSpec.scala :include: recover -In this example, if the actor replied with a ``akka.actor.Status.Failure`` containing the ``ArithmeticException``, our ``Future`` would have a result of 0. The ``recover`` method works very similarly to the standard try/catch blocks, so multiple ``Exception``\s can be handled in this manner, and if an ``Exception`` is not handled this way it will be behave as if we hadn't used the ``recover`` method. +In this example, if the actor replied with a ``akka.actor.Status.Failure`` containing the ``ArithmeticException``, +our ``Future`` would have a result of 0. The ``recover`` method works very similarly to the standard try/catch blocks, +so multiple ``Exception``\s can be handled in this manner, and if an ``Exception`` is not handled this way +it will behave as if we hadn't used the ``recover`` method. From 6afed30d430e29f99b7f34040e6e6c8c0cccdcc6 Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 24 Jan 2012 10:45:18 +0100 Subject: [PATCH 128/152] add Java FSM example and reST, see #1428 --- akka-docs/general/actors.rst | 14 +- .../code/akka/docs/actor/FSMDocTest.scala | 8 + .../code/akka/docs/actor/FSMDocTestBase.java | 174 ++++++++++++++++++ akka-docs/java/fsm.rst | 79 ++++++++ akka-docs/java/index.rst | 1 + akka-docs/scala/fsm.rst | 5 +- 6 files changed, 272 insertions(+), 9 deletions(-) create mode 100644 akka-docs/java/code/akka/docs/actor/FSMDocTest.scala create mode 100644 akka-docs/java/code/akka/docs/actor/FSMDocTestBase.java create mode 100644 akka-docs/java/fsm.rst diff --git a/akka-docs/general/actors.rst b/akka-docs/general/actors.rst index 952b1b08e4..4d8f2c8096 100644 --- a/akka-docs/general/actors.rst +++ b/akka-docs/general/actors.rst @@ -32,13 +32,13 @@ State Actor objects will typically contain some variables which reflect possible states the actor may be in. This can be an explicit state machine (e.g. using -the :ref:`fsm` module), or it could be a counter, set of listeners, pending -requests, etc. These data are what make an actor valuable, and they must be -protected from corruption by other actors. The good news is that Akka actors -conceptually each have their own light-weight thread, which is completely -shielded from the rest of the system. This means that instead of having to -synchronize access using locks you can just write your actor code without -worrying about concurrency at all. +the :ref:`fsm-scala` module), or it could be a counter, set of listeners, +pending requests, etc. These data are what make an actor valuable, and they +must be protected from corruption by other actors. The good news is that Akka +actors conceptually each have their own light-weight thread, which is +completely shielded from the rest of the system. This means that instead of +having to synchronize access using locks you can just write your actor code +without worrying about concurrency at all. Behind the scenes Akka will run sets of actors on sets of real threads, where typically many actors share one thread, and subsequent invocations of one actor diff --git a/akka-docs/java/code/akka/docs/actor/FSMDocTest.scala b/akka-docs/java/code/akka/docs/actor/FSMDocTest.scala new file mode 100644 index 0000000000..11bb542808 --- /dev/null +++ b/akka-docs/java/code/akka/docs/actor/FSMDocTest.scala @@ -0,0 +1,8 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.docs.actor + +import org.scalatest.junit.JUnitSuite + +class FSMDocTest extends FSMDocTestBase with JUnitSuite \ No newline at end of file diff --git a/akka-docs/java/code/akka/docs/actor/FSMDocTestBase.java b/akka-docs/java/code/akka/docs/actor/FSMDocTestBase.java new file mode 100644 index 0000000000..981cac15b1 --- /dev/null +++ b/akka-docs/java/code/akka/docs/actor/FSMDocTestBase.java @@ -0,0 +1,174 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.docs.actor; + +//#imports-data +import java.util.ArrayList; +import java.util.List; +import akka.actor.ActorRef; +//#imports-data + +//#imports-actor +import akka.event.LoggingAdapter; +import akka.event.Logging; +import akka.actor.UntypedActor; +//#imports-actor + +import akka.actor.ActorSystem; +import akka.actor.Props; +import akka.testkit.TestProbe; + +public class FSMDocTestBase { + + //#data + public static final class SetTarget { + final ActorRef ref; + public SetTarget(ActorRef ref) { + this.ref = ref; + } + } + + public static final class Queue { + final Object o; + public Queue(Object o) { + this.o = o; + } + } + + public static final Object flush = new Object(); + + public static final class Batch { + final List objects; + public Batch(List objects) { + this.objects = objects; + } + } + //#data + + //#base + static abstract class MyFSMBase extends UntypedActor { + + /* + * This is the mutable state of this state machine. + */ + protected enum State { IDLE, ACTIVE; } + private State state = State.IDLE; + private ActorRef target; + private List queue; + + /* + * Then come all the mutator methods: + */ + protected void init(ActorRef target) { + this.target = target; + queue = new ArrayList(); + } + + protected void setState(State s) { + if (state != s) { + transition(state, s); + state = s; + } + } + + protected void enqueue(Object o) { + if (queue != null) queue.add(o); + } + + protected List drainQueue() { + final List q = queue; + if (q == null) throw new IllegalStateException("drainQueue(): not yet initialized"); + queue = new ArrayList(); + return q; + } + + /* + * Here are the interrogation methods: + */ + protected boolean isInitialized() { + return target != null; + } + + protected State getState() { + return state; + } + + protected ActorRef getTarget() { + if (target == null) throw new IllegalStateException("getTarget(): not yet initialized"); + return target; + } + + /* + * And finally the callbacks (only one in this example: react to state change) + */ + abstract protected void transition(State old, State next); + } + //#base + + //#actor + static public class MyFSM extends MyFSMBase { + + private final LoggingAdapter log = Logging.getLogger(getContext().system(), this); + + @Override + public void onReceive(Object o) { + + if (getState() == State.IDLE) { + + if (o instanceof SetTarget) + init(((SetTarget) o).ref); + + else whenUnhandled(o); + + } else if (getState() == State.ACTIVE) { + + if (o == flush) + setState(State.IDLE); + + else whenUnhandled(o); + } + } + + @Override + public void transition(State old, State next) { + if (old == State.ACTIVE) { + getTarget().tell(new Batch(drainQueue())); + } + } + + private void whenUnhandled(Object o) { + if (o instanceof Queue && isInitialized()) { + enqueue(((Queue) o).o); + setState(State.ACTIVE); + + } else { + log.warning("received unknown message {} in state {}", o, getState()); + } + } + } + //#actor + + ActorSystem system = ActorSystem.create(); + + @org.junit.Test + public void mustBunch() { + final ActorRef buncher = system.actorOf(new Props(MyFSM.class)); + final TestProbe probe = new TestProbe(system); + buncher.tell(new SetTarget(probe.ref())); + buncher.tell(new Queue(1)); + buncher.tell(new Queue(2)); + buncher.tell(flush); + buncher.tell(new Queue(3)); + final Batch b = probe.expectMsgClass(Batch.class); + assert b.objects.size() == 2; + assert b.objects.contains(1); + assert b.objects.contains(2); + } + + @org.junit.After + public void cleanup() { + system.shutdown(); + } + +} diff --git a/akka-docs/java/fsm.rst b/akka-docs/java/fsm.rst new file mode 100644 index 0000000000..d66627d416 --- /dev/null +++ b/akka-docs/java/fsm.rst @@ -0,0 +1,79 @@ +.. _fsm-java: + +########################################### +Building Finite State Machine Actors (Java) +########################################### + +.. sidebar:: Contents + + .. contents:: :local: + +Overview +======== + +The FSM (Finite State Machine) pattern is best described in the `Erlang design +principles +`_. +In short, it can be seen as a set of relations of the form: + + **State(S) x Event(E) -> Actions (A), State(S')** + +These relations are interpreted as meaning: + + *If we are in state S and the event E occurs, we should perform the actions A + and make a transition to the state S'.* + +While the Scala programming language enables the formulation of a nice internal +DSL (domain specific language) for formulating finite state machines (see +:ref:`fsm-scala`), Java’s verbosity does not lend itself well to the same +approach. This chapter describes ways to effectively achieve the same +separation of concerns through self-discipline. + +How State should be Handled +=========================== + +All mutable fields (or transitively mutable data structures) referenced by the +FSM actor’s implementation should be collected in one place and only mutated +using a small well-defined set of methods. One way to achieve this is to +assemble all mutable state in a superclass which keeps it private and offers +protected methods for mutating it. + +.. includecode:: code/akka/docs/actor/FSMDocTestBase.java#imports-data + +.. includecode:: code/akka/docs/actor/FSMDocTestBase.java#base + +The benefit of this approach is that state changes can be acted upon in one +central place, which makes it impossible to forget inserting code for reacting +to state transitions when adding to the FSM’s machinery. + +Message Buncher Example +======================= + +The base class shown above is designed to support a similar example as for the +Scala FSM documentation: an actor which receives and queues messages, to be +delivered in batches to a configurable target actor. The messages involved are: + +.. includecode:: code/akka/docs/actor/FSMDocTestBase.java#data + +This actor has only the two states ``IDLE`` and ``ACTIVE``, making their +handling quite straight-forward in the concrete actor derived from the base +class: + +.. includecode:: code/akka/docs/actor/FSMDocTestBase.java#imports-actor + +.. includecode:: code/akka/docs/actor/FSMDocTestBase.java#actor + +The trick here is to factor out common functionality like :meth:`whenUnhandled` +and :meth:`transition` in order to obtain a few well-defined points for +reacting to change or insert logging. + +State-Centric vs. Event-Centric +=============================== + +In the example above, the subjective complexity of state and events was roughly +equal, making it a matter of taste whether to choose primary dispatch on +either; in the example a state-based dispatch was chosen. Depending on how +evenly the matrix of possible states and events is populated, it may be more +practical to handle different events first and distinguish the states in the +second tier. An example would be a state machine which has a multitude of +internal states but handles only very few distinct events. diff --git a/akka-docs/java/index.rst b/akka-docs/java/index.rst index 4b0226fc35..319dbab302 100644 --- a/akka-docs/java/index.rst +++ b/akka-docs/java/index.rst @@ -21,4 +21,5 @@ Java API stm agents transactors + fsm extending-akka diff --git a/akka-docs/scala/fsm.rst b/akka-docs/scala/fsm.rst index 7b3d136ae4..618381901c 100644 --- a/akka-docs/scala/fsm.rst +++ b/akka-docs/scala/fsm.rst @@ -1,4 +1,4 @@ -.. _fsm: +.. _fsm-scala: ### FSM @@ -21,7 +21,8 @@ A FSM can be described as a set of relations of the form: These relations are interpreted as meaning: - *If we are in state S and the event E occurs, we should perform the actions A and make a transition to the state S'.* + *If we are in state S and the event E occurs, we should perform the actions A + and make a transition to the state S'.* A Simple Example ================ From 4c6eba57db9c1d77d94ba331eb84491c19686571 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Sun, 22 Jan 2012 16:33:20 +0100 Subject: [PATCH 129/152] DOC: Added new fault handling sample. See #1574 --- .../docs/actor/FaultHandlingDocSample.scala | 271 ++++++++++++++++++ akka-docs/scala/fault-tolerance.rst | 40 ++- 2 files changed, 309 insertions(+), 2 deletions(-) create mode 100644 akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSample.scala diff --git a/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSample.scala b/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSample.scala new file mode 100644 index 0000000000..cb400629d6 --- /dev/null +++ b/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSample.scala @@ -0,0 +1,271 @@ +/** + * Copyright (C) 2009-2012 Typesafe Inc. + */ +package akka.docs.actor + +//#all +import akka.actor._ +import akka.actor.SupervisorStrategy._ +import akka.util.duration._ +import akka.util.Duration +import akka.util.Timeout +import akka.event.LoggingReceive +import akka.pattern.ask +import com.typesafe.config.ConfigFactory + +/** + * Runs the sample + */ +object FaultHandlingDocSample extends App { + import Worker._ + import CounterService._ + + val config = ConfigFactory.parseString(""" + akka.loglevel = DEBUG + akka.actor.debug { + receive = on + lifecycle = on + } + """) + + val system = ActorSystem("FaultToleranceSample", config) + val worker = system.actorOf(Props[Worker], name = "worker") + + system.actorOf(Props(new Actor with ActorLogging { + // If we don't get any progress within 15 seconds then the service is unavailable + context.setReceiveTimeout(15 seconds) + worker ! Start + + def receive = { + case CurrentCount(key, count) ⇒ + log.info("Current count for [{}] is [{}]", key, count) + if (count > 50) { + log.info("That's enough, shutting down") + system.shutdown() + } + + case ReceiveTimeout ⇒ + // No progress within 15 seconds, ServiceUnavailable + log.error("Shutting down due to unavailable service") + system.shutdown() + } + })) + +} + +object Worker { + // Messages + case object Start + case object Do +} + +/** + * Worker performs some work when it receives the `Start` message. + * It will continuously notify the sender of the `Start` message + * of current progress. The `Worker` supervise the `CounterService`. + */ +class Worker extends Actor with ActorLogging { + import Worker._ + import CounterService._ + implicit def system = context.system + implicit val askTimeout = Timeout(5 seconds) + + // Stop the CounterService child if it throws ServiceUnavailable + override val supervisorStrategy = OneForOneStrategy() { + case _: CounterService.ServiceUnavailable ⇒ Stop + } + + // The sender of the initial Start message will continuously be notified about progress + var progressListener: ActorRef = _ + val counterService = context.actorOf(Props[CounterService], name = "counter") + + def receive = LoggingReceive(this) { + case Start if progressListener eq null ⇒ + progressListener = sender + context.system.scheduler.schedule(Duration.Zero, 1 second, self, Do) + + case Do ⇒ + counterService ! Increment(1) + counterService ! Increment(1) + counterService ! Increment(1) + + // Send current count to the initial sender + (counterService ? GetCurrentCount).pipeTo(progressListener) + } +} + +object CounterService { + // Messages + case class Increment(n: Int) + case object GetCurrentCount + case class CurrentCount(key: String, count: Long) + case object Reconnect + class ServiceUnavailable(msg: String) extends RuntimeException(msg) +} + +/** + * Adds the value received in `Increment` message to a persistent + * counter. Replies with `CurrentCount` when it is asked for `CurrentCount`. + * `CounterService` supervise `Storage` and `Counter`. + */ +class CounterService extends Actor { + import CounterService._ + import Counter._ + import Storage._ + implicit def system = context.system + + // Restart the storage child when StorageException is thrown. + // After 3 restarts within 5 seconds it will be stopped. + override val supervisorStrategy = OneForOneStrategy(maxNrOfRetries = 3, withinTimeRange = 5 seconds) { + case _: Storage.StorageException ⇒ Restart + } + + val key = context.self.path.name + var storage: Option[ActorRef] = None + var counter: Option[ActorRef] = None + var backlog = IndexedSeq.empty[Any] + val MaxBacklog = 10000 + + override def preStart() { + initStorage() + } + + def initStorage() { + // The child storage is restarted in case of failure, but after 3 restarts, + // and still failing it will be stopped. Better to back-off than continuously + // failing. When it has been stopped we will schedule a Reconnect after a delay. + // Watch the child so we receive Terminated message when it has been terminated. + storage = Some(context.watch(context.actorOf(Props[Storage], name = "storage"))) + // Tell the counter, if any, to use the new storage + counter foreach { _ ! UseStorage(storage) } + // We need the initial value to be able to operate + storage foreach { _ ! Get(key) } + } + + def receive = LoggingReceive(this) { + + case Entry(k, v) if k == key && counter == None ⇒ + // Reply from Storage of the initial value, now we can create the Counter + val c = context.actorOf(Props(new Counter(key, v))) + counter = Some(c) + // Tell the counter to use current storage + c ! UseStorage(storage) + // and send the buffered backlog to the counter + backlog foreach { c ! _ } + backlog = IndexedSeq.empty + + case msg @ Increment(n) ⇒ forwardOrPlaceInBacklow(msg) + + case msg @ GetCurrentCount ⇒ forwardOrPlaceInBacklow(msg) + + case Terminated(actorRef) if Some(actorRef) == storage ⇒ + // After 3 restarts the storage child is stopped. + // We receive Terminated because we watch the child, see initStorage. + storage = None + // Tell the counter that there is no storage for the moment + counter foreach { _ ! UseStorage(None) } + // Try to re-establish storage after while + context.system.scheduler.scheduleOnce(10 seconds, self, Reconnect) + + case Reconnect ⇒ + // Re-establish storage after the scheduled delay + initStorage() + } + + def forwardOrPlaceInBacklow(msg: Any) { + // We need the initial value from storage before we can start delegate to the counter. + // Before that we place the messages in a backlog, to be sent to the counter when + // it is initialized. + counter match { + case Some(c) ⇒ c forward msg + case None ⇒ + if (backlog.size >= MaxBacklog) + throw new ServiceUnavailable("CounterService not available, lack of initial value") + backlog = backlog :+ msg + } + } + +} + +object Counter { + // Messages + case class UseStorage(storage: Option[ActorRef]) +} + +/** + * The in memory count variable that will send current + * value to the `Storage`, if there is any storage + * available at the moment. + */ +class Counter(key: String, initialValue: Long) extends Actor { + import Counter._ + import CounterService._ + import Storage._ + implicit def system = context.system + + var count = initialValue + var storage: Option[ActorRef] = None + + def receive = LoggingReceive(this) { + case UseStorage(s) ⇒ + storage = s + storeCount() + + case Increment(n) ⇒ + count += n + storeCount() + + case GetCurrentCount ⇒ + sender ! CurrentCount(key, count) + + } + + def storeCount() { + // Delegate dangerous work, to protect our valuable state. + // We can continue without storage. + storage foreach { _ ! Store(Entry(key, count)) } + } + +} + +object Storage { + // Messages + case class Store(entry: Entry) + case class Get(key: String) + case class Entry(key: String, value: Long) + class StorageException(msg: String) extends RuntimeException(msg) +} + +/** + * Saves key/value pairs to persistent storage when receiving `Store` message. + * Replies with current value when receiving `Get` message. + * Will throw StorageException if the underlying data store is out of order. + */ +class Storage extends Actor { + import Storage._ + implicit def system = context.system + + val db = DummyDB + + def receive = LoggingReceive(this) { + case Store(Entry(key, count)) ⇒ db.save(key, count) + case Get(key) ⇒ sender ! Entry(key, db.load(key).getOrElse(0L)) + } +} + +object DummyDB { + import Storage.StorageException + var db = Map[String, Long]() + + @throws(classOf[StorageException]) + def save(key: String, value: Long): Unit = synchronized { + if (11 <= value && value <= 14) throw new StorageException("Simulated store failure " + value) + db += (key -> value) + } + + @throws(classOf[StorageException]) + def load(key: String): Option[Long] = synchronized { + db.get(key) + } +} +//#all diff --git a/akka-docs/scala/fault-tolerance.rst b/akka-docs/scala/fault-tolerance.rst index 4c9925c947..4edada1a73 100644 --- a/akka-docs/scala/fault-tolerance.rst +++ b/akka-docs/scala/fault-tolerance.rst @@ -12,9 +12,26 @@ children, and as such each actor defines fault handling supervisor strategy. This strategy cannot be changed afterwards as it is an integral part of the actor system’s structure. +Fault Handling in Practice +-------------------------- + +First, let us look at a sample that illustrates one way to handle data store errors, +which is a typical source of failure in real world applications. Of course it depends +on the actual application what is possible to do when the data store is unavailable, +but in this sample we use a best effort re-connect approach. + +Read the following source code. The inlined comments explain the different pieces of +the fault handling and why they are added. It is also highly recommended to run this +sample as it is easy to follow the log output to understand what is happening in runtime. + +.. includecode:: code/akka/docs/actor/FaultHandlingDocSample.scala#all + Creating a Supervisor Strategy ------------------------------ +The following sections explain the fault handling mechanism and alternatives +in more depth. + For the sake of demonstration let us consider the following strategy: .. includecode:: code/akka/docs/actor/FaultHandlingDocSpec.scala @@ -34,8 +51,27 @@ The match statement which forms the bulk of the body is of type ``Decider``, which is a ``PartialFunction[Throwable, Action]``. This is the piece which maps child failure types to their corresponding actions. -Practical Application ---------------------- +Default Supervisor Strategy +--------------------------- + +``Escalate`` is used if the defined strategy doesn't cover the exception that was thrown. + +When the supervisor strategy is not defined for an actor the following +exceptions are handled by default:: + + OneForOneStrategy { + case _: ActorInitializationException ⇒ Stop + case _: ActorKilledException ⇒ Stop + case _: Exception ⇒ Restart + case _ ⇒ Escalate + } + +If the exception escalate all the way up to the root guardian it will handle it +in the same way as the default strategy defined above. + + +Test Application +---------------- The following section shows the effects of the different actions in practice, wherefor a test setup is needed. First off, we need a suitable supervisor: From 28dfdaba32b6b769f0ef52389793601d2ed843c9 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 24 Jan 2012 15:03:32 +0100 Subject: [PATCH 130/152] Correcting after review --- .../scala/code/akka/docs/future/FutureDocSpec.scala | 10 ++++++++-- akka-docs/scala/futures.rst | 6 +++--- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala b/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala index d1e9532548..175fc08ff5 100644 --- a/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala +++ b/akka-docs/scala/code/akka/docs/future/FutureDocSpec.scala @@ -131,6 +131,12 @@ class FutureDocSpec extends AkkaSpec { val future2 = future1.filter(_ % 2 == 0) val result = Await.result(future2, 1 second) result must be(4) + + val failedFilter = future1.filter(_ % 2 == 1).recover { + case m: MatchError ⇒ 0 //When filter fails, it will have a MatchError + } + val result2 = Await.result(failedFilter, 1 second) + result2 must be(0) //Can only be 0 when there was a MatchError //#filter } @@ -307,8 +313,8 @@ class FutureDocSpec extends AkkaSpec { def doSomethingOnFailure(t: Throwable) = () //#onComplete future onComplete { - case Right(result) ⇒ doSomethingOnSuccess(result) //Right == Success - case Left(failure) ⇒ doSomethingOnFailure(failure) //Left == Failure + case Right(result) ⇒ doSomethingOnSuccess(result) + case Left(failure) ⇒ doSomethingOnFailure(failure) } //#onComplete Await.result(future, 1 second) must be("foo") diff --git a/akka-docs/scala/futures.rst b/akka-docs/scala/futures.rst index 5d0baa5ca7..c46db30927 100644 --- a/akka-docs/scala/futures.rst +++ b/akka-docs/scala/futures.rst @@ -164,8 +164,8 @@ But it may be faster to use ``traverse`` as it doesn't have to create an interme Then there's a method that's called ``fold`` that takes a start-value, a sequence of ``Future``\s and a function from the type of the start-value and the type of the futures and returns something with the same type as the start-value, -and then applies the function to all elements in the sequence of futures, non-blockingly, -the execution will run on the Thread of the last completing Future in the sequence. +and then applies the function to all elements in the sequence of futures, asynchronously, +the execution will start when the last of the Futures is completed. .. includecode:: code/akka/docs/future/FutureDocSpec.scala :include: fold @@ -180,7 +180,7 @@ as the start-value, you can use ``reduce``, it works like this: .. includecode:: code/akka/docs/future/FutureDocSpec.scala :include: reduce -Same as with ``fold``, the execution will be done by the Thread that completes the last of the Futures, ` +Same as with ``fold``, the execution will be done asynchronously when the last of the Future is completed,` you can also parallelize it by chunking your futures into sub-sequences and reduce them, and then reduce the reduced results again. Callbacks From cd608301f8785ff9f742984e4f975449e1336027 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 24 Jan 2012 16:02:21 +0100 Subject: [PATCH 131/152] Commented out Schoir plugin. It has bad dependency to log4j 1.2.15. See #1721 --- project/AkkaBuild.scala | 4 ++-- project/plugins.sbt | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 9e5e773911..f32987fb45 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -8,7 +8,7 @@ import sbt._ import sbt.Keys._ import com.typesafe.sbtmultijvm.MultiJvmPlugin import com.typesafe.sbtmultijvm.MultiJvmPlugin.{ MultiJvm, extraOptions, jvmOptions, scalatestOptions } -import com.typesafe.schoir.SchoirPlugin.schoirSettings +//import com.typesafe.schoir.SchoirPlugin.schoirSettings import com.typesafe.sbtscalariform.ScalariformPlugin import com.typesafe.sbtscalariform.ScalariformPlugin.ScalariformKeys import java.lang.Boolean.getBoolean @@ -71,7 +71,7 @@ object AkkaBuild extends Build { id = "akka-remote", base = file("akka-remote"), dependencies = Seq(actor, actorTests % "test->test", testkit % "test->test"), - settings = defaultSettings ++ multiJvmSettings ++ schoirSettings ++ Seq( + settings = defaultSettings ++ multiJvmSettings ++ /*schoirSettings ++*/ Seq( libraryDependencies ++= Dependencies.remote, // disable parallel tests parallelExecution in Test := false, diff --git a/project/plugins.sbt b/project/plugins.sbt index 024f70877c..899db6307f 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -3,7 +3,7 @@ resolvers += Classpaths.typesafeResolver addSbtPlugin("com.typesafe.sbtmultijvm" % "sbt-multi-jvm" % "0.1.9") -addSbtPlugin("com.typesafe.schoir" % "schoir" % "0.1.1") +//addSbtPlugin("com.typesafe.schoir" % "schoir" % "0.1.1") addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse" % "1.5.0") From 5483dacabaff767384a01b3a71db048917afb491 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 24 Jan 2012 16:31:20 +0100 Subject: [PATCH 132/152] Adding Future docs for Java API --- .../akka/docs/future/FutureDocTestBase.java | 177 +++++++++++++++--- akka-docs/java/futures.rst | 134 ++++++++++--- 2 files changed, 259 insertions(+), 52 deletions(-) diff --git a/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java b/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java index d2adfaf5fa..e642047709 100644 --- a/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java +++ b/akka-docs/java/code/akka/docs/future/FutureDocTestBase.java @@ -4,6 +4,9 @@ package akka.docs.future; //#imports1 +import akka.dispatch.Promise; +import akka.japi.Procedure; +import akka.japi.Procedure2; import akka.util.Timeout; import akka.dispatch.Await; import akka.dispatch.Future; @@ -196,17 +199,19 @@ public class FutureDocTestBase { Iterable> listOfFutureInts = source; // now we have a Future[Iterable[Integer]] - Future> futureListOfInts = sequence(listOfFutureInts, system.dispatcher()); + Future> futureListOfInts = + sequence(listOfFutureInts, system.dispatcher()); // Find the sum of the odd numbers - Future futureSum = futureListOfInts.map(new Function, Long>() { - public Long apply(Iterable ints) { - long sum = 0; - for (Integer i : ints) - sum += i; - return sum; - } - }); + Future futureSum = futureListOfInts.map( + new Function, Long>() { + public Long apply(Iterable ints) { + long sum = 0; + for (Integer i : ints) + sum += i; + return sum; + } + }); long result = Await.result(futureSum, Duration.create(1, SECONDS)); //#sequence @@ -219,18 +224,20 @@ public class FutureDocTestBase { //Just a sequence of Strings Iterable listStrings = Arrays.asList("a", "b", "c"); - Future> futureResult = traverse(listStrings, new Function>() { - public Future apply(final String r) { - return future(new Callable() { - public String call() { - return r.toUpperCase(); - } - }, system.dispatcher()); - } - }, system.dispatcher()); + Future> futureResult = traverse(listStrings, + new Function>() { + public Future apply(final String r) { + return future(new Callable() { + public String call() { + return r.toUpperCase(); + } + }, system.dispatcher()); + } + }, system.dispatcher()); //Returns the sequence of strings as upper case - Iterable result = Await.result(futureResult, Duration.create(1, SECONDS)); + Iterable result = + Await.result(futureResult, Duration.create(1, SECONDS)); assertEquals(Arrays.asList("A", "B", "C"), result); //#traverse } @@ -246,11 +253,12 @@ public class FutureDocTestBase { Iterable> futures = source; //Start value is the empty string - Future resultFuture = fold("", futures, new Function2() { - public String apply(String r, String t) { - return r + t; //Just concatenate - } - }, system.dispatcher()); + Future resultFuture = fold("", futures, + new Function2() { + public String apply(String r, String t) { + return r + t; //Just concatenate + } + }, system.dispatcher()); String result = Await.result(resultFuture, Duration.create(1, SECONDS)); //#fold @@ -267,11 +275,12 @@ public class FutureDocTestBase { //A sequence of Futures, in this case Strings Iterable> futures = source; - Future resultFuture = reduce(futures, new Function2() { - public Object apply(Object r, String t) { - return r + t; //Just concatenate - } - }, system.dispatcher()); + Future resultFuture = reduce(futures, + new Function2() { + public Object apply(Object r, String t) { + return r + t; //Just concatenate + } + }, system.dispatcher()); Object result = Await.result(resultFuture, Duration.create(1, SECONDS)); //#reduce @@ -279,6 +288,116 @@ public class FutureDocTestBase { assertEquals("ab", result); } + @Test public void useSuccessfulAndFailed() { + //#successful + Future future = Futures.successful("Yay!", system.dispatcher()); + //#successful + //#failed + Future otherFuture = + Futures.failed(new IllegalArgumentException("Bang!"), system.dispatcher()); + //#failed + Object result = Await.result(future, Duration.create(1, SECONDS)); + assertEquals("Yay!",result); + Throwable result2 = Await.result(otherFuture.failed(), Duration.create(1, SECONDS)); + assertEquals("Bang!",result2.getMessage()); + } + + @Test public void useFilter() { + //#filter + Future future1 = Futures.successful(4, system.dispatcher()); + Future successfulFilter = + future1.filter(new Function() { + public Boolean apply(Integer i) { return i % 2 == 0; } + }); + + Future failedFilter = + future1.filter(new Function() { + public Boolean apply(Integer i) { return i % 2 != 0; } + }); + //When filter fails, the returned Future will be failed with a scala.MatchError + //#filter + } + + @Test public void useOnSuccessOnFailureAndOnComplete() { + { + Future future = Futures.successful("foo", system.dispatcher()); + //#onSuccess + future.onSuccess(new Procedure() { + public void apply(String result) { + if ("bar" == result) { + //Do something if it resulted in "bar" + } else { + //Do something if it was some other String + } + } + }); + //#onSuccess + } + { + Future future = + Futures.failed(new IllegalStateException("OHNOES"), system.dispatcher()); + //#onFailure + future.onFailure( new Procedure() { + public void apply(Throwable failure) { + if (failure instanceof IllegalStateException) { + //Do something if it was this particular failure + } else { + //Do something if it was some other failure + } + } + }); + //#onFailure + } + { + Future future = Futures.successful("foo", system.dispatcher()); + //#onComplete + future.onComplete(new Procedure2() { + public void apply(Throwable failure, String result) { + if (failure != null) { + //We got a failure, handle it here + } else { + // We got a result, do something with it + } + } + }); + //#onComplete + } + } + + @Test public void useOrAndZip(){ + { + //#zip + Future future1 = Futures.successful("foo", system.dispatcher()); + Future future2 = Futures.successful("bar", system.dispatcher()); + Future future3 = + future1.zip(future2).map(new Function, String>() { + public String apply(scala.Tuple2 zipped) { + return zipped._1() + " " + zipped._2(); + } + }); + + String result = Await.result(future3, Duration.create(1, SECONDS)); + assertEquals("foo bar", result); + //#zip + } + + { + //#or + Future future1 = + Futures.failed(new IllegalStateException("OHNOES1"), system.dispatcher()); + Future future2 = + Futures.failed(new IllegalStateException("OHNOES2"), system.dispatcher()); + Future future3 = + Futures.successful("bar", system.dispatcher()); + Future future4 = + future1.or(future2).or(future3); // Will have "bar" in this case + String result = Await.result(future4, Duration.create(1, SECONDS)); + assertEquals("bar", result); + //#or + } + + } + public static class MyActor extends UntypedActor { public void onReceive(Object message) { if (message instanceof String) { diff --git a/akka-docs/java/futures.rst b/akka-docs/java/futures.rst index 3a482ab2dc..e9b743535a 100644 --- a/akka-docs/java/futures.rst +++ b/akka-docs/java/futures.rst @@ -10,86 +10,138 @@ Futures (Java) Introduction ------------ -In Akka, a `Future `_ is a data structure used to retrieve the result of some concurrent operation. This operation is usually performed by an ``Actor`` or by the ``Dispatcher`` directly. This result can be accessed synchronously (blocking) or asynchronously (non-blocking). +In Akka, a `Future `_ is a data structure used +to retrieve the result of some concurrent operation. This operation is usually performed by an ``Actor`` or +by the ``Dispatcher`` directly. This result can be accessed synchronously (blocking) or asynchronously (non-blocking). Use with Actors --------------- -There are generally two ways of getting a reply from an ``UntypedActor``: the first is by a sent message (``actorRef.tell(msg)``), which only works if the original sender was an ``UntypedActor``) and the second is through a ``Future``. +There are generally two ways of getting a reply from an ``UntypedActor``: the first is by a sent message (``actorRef.tell(msg)``), +which only works if the original sender was an ``UntypedActor``) and the second is through a ``Future``. -Using the ``ActorRef``\'s ``ask`` method to send a message will return a Future. To wait for and retrieve the actual result the simplest method is: +Using the ``ActorRef``\'s ``ask`` method to send a message will return a Future. +To wait for and retrieve the actual result the simplest method is: .. includecode:: code/akka/docs/future/FutureDocTestBase.java :include: imports1,ask-blocking -This will cause the current thread to block and wait for the ``UntypedActor`` to 'complete' the ``Future`` with it's reply. Blocking is discouraged though as it can cause performance problem. -The blocking operations are located in ``Await.result`` and ``Await.ready`` to make it easy to spot where blocking occurs. Alternatives to blocking are discussed further within this documentation. -Also note that the ``Future`` returned by an ``UntypedActor`` is a ``Future`` since an ``UntypedActor`` is dynamic. That is why the cast to ``String`` is used in the above sample. +This will cause the current thread to block and wait for the ``UntypedActor`` to 'complete' the ``Future`` with it's reply. +Blocking is discouraged though as it can cause performance problem. +The blocking operations are located in ``Await.result`` and ``Await.ready`` to make it easy to spot where blocking occurs. +Alternatives to blocking are discussed further within this documentation. +Also note that the ``Future`` returned by an ``UntypedActor`` is a ``Future`` since an ``UntypedActor`` is dynamic. +That is why the cast to ``String`` is used in the above sample. Use Directly ------------ -A common use case within Akka is to have some computation performed concurrently without needing the extra utility of an ``UntypedActor``. If you find yourself creating a pool of ``UntypedActor``\s for the sole reason of performing a calculation in parallel, there is an easier (and faster) way: +A common use case within Akka is to have some computation performed concurrently without needing +the extra utility of an ``UntypedActor``. If you find yourself creating a pool of ``UntypedActor``\s for the sole reason +of performing a calculation in parallel, there is an easier (and faster) way: .. includecode:: code/akka/docs/future/FutureDocTestBase.java :include: imports2,future-eval -In the above code the block passed to ``future`` will be executed by the default ``Dispatcher``, with the return value of the block used to complete the ``Future`` (in this case, the result would be the string: "HelloWorld"). Unlike a ``Future`` that is returned from an ``UntypedActor``, this ``Future`` is properly typed, and we also avoid the overhead of managing an ``UntypedActor``. +In the above code the block passed to ``future`` will be executed by the default ``Dispatcher``, +with the return value of the block used to complete the ``Future`` (in this case, the result would be the string: "HelloWorld"). +Unlike a ``Future`` that is returned from an ``UntypedActor``, this ``Future`` is properly typed, +and we also avoid the overhead of managing an ``UntypedActor``. + +You can also create already completed Futures using the ``Futures`` class, which can be either successes: + +.. includecode:: code/akka/docs/future/FutureDocTestBase.java + :include: successful + +Or failures: + +.. includecode:: code/akka/docs/future/FutureDocTestBase.java + :include: failed Functional Futures ------------------ -Akka's ``Future`` has several monadic methods that are very similar to the ones used by ``Scala``'s collections. These allow you to create 'pipelines' or 'streams' that the result will travel through. +Akka's ``Future`` has several monadic methods that are very similar to the ones used by ``Scala``'s collections. +These allow you to create 'pipelines' or 'streams' that the result will travel through. Future is a Monad ^^^^^^^^^^^^^^^^^ -The first method for working with ``Future`` functionally is ``map``. This method takes a ``Function`` which performs some operation on the result of the ``Future``, and returning a new result. The return value of the ``map`` method is another ``Future`` that will contain the new result: +The first method for working with ``Future`` functionally is ``map``. This method takes a ``Function`` which performs +some operation on the result of the ``Future``, and returning a new result. +The return value of the ``map`` method is another ``Future`` that will contain the new result: .. includecode:: code/akka/docs/future/FutureDocTestBase.java :include: imports2,map -In this example we are joining two strings together within a Future. Instead of waiting for f1 to complete, we apply our function that calculates the length of the string using the ``map`` method. Now we have a second Future, f2, that will eventually contain an ``Integer``. When our original ``Future``, f1, completes, it will also apply our function and complete the second Future with it's result. When we finally ``get`` the result, it will contain the number 10. Our original Future still contains the string "HelloWorld" and is unaffected by the ``map``. +In this example we are joining two strings together within a Future. Instead of waiting for f1 to complete, +we apply our function that calculates the length of the string using the ``map`` method. +Now we have a second Future, f2, that will eventually contain an ``Integer``. +When our original ``Future``, f1, completes, it will also apply our function and complete the second Future +with its result. When we finally ``get`` the result, it will contain the number 10. +Our original Future still contains the string "HelloWorld" and is unaffected by the ``map``. -Something to note when using these methods: if the ``Future`` is still being processed when one of these methods are called, it will be the completing thread that actually does the work. If the ``Future`` is already complete though, it will be run in our current thread. For example: +Something to note when using these methods: if the ``Future`` is still being processed when one of these methods are called, +it will be the completing thread that actually does the work. +If the ``Future`` is already complete though, it will be run in our current thread. For example: .. includecode:: code/akka/docs/future/FutureDocTestBase.java :include: map2 -The original ``Future`` will take at least 0.1 second to execute now, which means it is still being processed at the time we call ``map``. The function we provide gets stored within the ``Future`` and later executed automatically by the dispatcher when the result is ready. +The original ``Future`` will take at least 0.1 second to execute now, which means it is still being processed at +the time we call ``map``. The function we provide gets stored within the ``Future`` and later executed automatically +by the dispatcher when the result is ready. If we do the opposite: .. includecode:: code/akka/docs/future/FutureDocTestBase.java :include: map3 -Our little string has been processed long before our 0.1 second sleep has finished. Because of this, the dispatcher has moved onto other messages that need processing and can no longer calculate the length of the string for us, instead it gets calculated in the current thread just as if we weren't using a ``Future``. +Our little string has been processed long before our 0.1 second sleep has finished. Because of this, +the dispatcher has moved onto other messages that need processing and can no longer calculate +the length of the string for us, instead it gets calculated in the current thread just as if we weren't using a ``Future``. -Normally this works quite well as it means there is very little overhead to running a quick function. If there is a possibility of the function taking a non-trivial amount of time to process it might be better to have this done concurrently, and for that we use ``flatMap``: +Normally this works quite well as it means there is very little overhead to running a quick function. +If there is a possibility of the function taking a non-trivial amount of time to process it might be better +to have this done concurrently, and for that we use ``flatMap``: .. includecode:: code/akka/docs/future/FutureDocTestBase.java :include: flat-map -Now our second Future is executed concurrently as well. This technique can also be used to combine the results of several Futures into a single calculation, which will be better explained in the following sections. +Now our second Future is executed concurrently as well. This technique can also be used to combine the results +of several Futures into a single calculation, which will be better explained in the following sections. + +If you need to do conditional propagation, you can use ``filter``: + +.. includecode:: code/akka/docs/future/FutureDocTestBase.java + :include: filter Composing Futures ^^^^^^^^^^^^^^^^^ -It is very often desirable to be able to combine different Futures with eachother, below are some examples on how that can be done in a non-blocking fashion. +It is very often desirable to be able to combine different Futures with each other, +below are some examples on how that can be done in a non-blocking fashion. .. includecode:: code/akka/docs/future/FutureDocTestBase.java :include: imports3,sequence -To better explain what happened in the example, ``Future.sequence`` is taking the ``Iterable>`` and turning it into a ``Future>``. We can then use ``map`` to work with the ``Iterable`` directly, and we aggregate the sum of the ``Iterable``. +To better explain what happened in the example, ``Future.sequence`` is taking the ``Iterable>`` +and turning it into a ``Future>``. We can then use ``map`` to work with the ``Iterable`` directly, +and we aggregate the sum of the ``Iterable``. -The ``traverse`` method is similar to ``sequence``, but it takes a sequence of ``A``s and applies a function from ``A`` to ``Future`` and returns a ``Future>``, enabling parallel ``map`` over the sequence, if you use ``Futures.future`` to create the ``Future``. +The ``traverse`` method is similar to ``sequence``, but it takes a sequence of ``A``s and applies a function from ``A`` to ``Future`` +and returns a ``Future>``, enabling parallel ``map`` over the sequence, if you use ``Futures.future`` to create the ``Future``. .. includecode:: code/akka/docs/future/FutureDocTestBase.java :include: imports4,traverse It's as simple as that! -Then there's a method that's called ``fold`` that takes a start-value, a sequence of ``Future``:s and a function from the type of the start-value, a timeout, and the type of the futures and returns something with the same type as the start-value, and then applies the function to all elements in the sequence of futures, non-blockingly, the execution will run on the Thread of the last completing Future in the sequence. +Then there's a method that's called ``fold`` that takes a start-value, +a sequence of ``Future``:s and a function from the type of the start-value, a timeout, +and the type of the futures and returns something with the same type as the start-value, +and then applies the function to all elements in the sequence of futures, non-blockingly, +the execution will be started when the last of the Futures is completed. .. includecode:: code/akka/docs/future/FutureDocTestBase.java :include: imports5,fold @@ -97,16 +149,52 @@ Then there's a method that's called ``fold`` that takes a start-value, a sequenc That's all it takes! -If the sequence passed to ``fold`` is empty, it will return the start-value, in the case above, that will be empty String. In some cases you don't have a start-value and you're able to use the value of the first completing Future in the sequence as the start-value, you can use ``reduce``, it works like this: +If the sequence passed to ``fold`` is empty, it will return the start-value, in the case above, that will be empty String. +In some cases you don't have a start-value and you're able to use the value of the first completing Future +in the sequence as the start-value, you can use ``reduce``, it works like this: .. includecode:: code/akka/docs/future/FutureDocTestBase.java :include: imports6,reduce -Same as with ``fold``, the execution will be done by the Thread that completes the last of the Futures, you can also parallize it by chunking your futures into sub-sequences and reduce them, and then reduce the reduced results again. +Same as with ``fold``, the execution will be started when the last of the Futures is completed, you can also parallelize +it by chunking your futures into sub-sequences and reduce them, and then reduce the reduced results again. This is just a sample of what can be done. +Callbacks +--------- + +Sometimes you just want to listen to a ``Future`` being completed, and react to that not by creating a new Future, but by side-effecting. +For this Akka supports ``onComplete``, ``onSuccess`` and ``onFailure``, of which the latter two are specializations of the first. + +.. includecode:: code/akka/docs/future/FutureDocTestBase.java + :include: onSuccess + +.. includecode:: code/akka/docs/future/FutureDocTestBase.java + :include: onFailure + +.. includecode:: code/akka/docs/future/FutureDocTestBase.java + :include: onComplete + +Auxiliary methods +----------------- + +``Future`` ``or`` combines 2 Futures into a new ``Future``, and will hold the successful value of the second ``Future` +if the first ``Future`` fails. + +.. includecode:: code/akka/docs/future/FutureDocTestBase.java + :include: or + +You can also combine two Futures into a new ``Future`` that will hold a tuple of the two Futures successful results, +using the ``zip`` operation. + +.. includecode:: code/akka/docs/future/FutureDocTestBase.java + :include: zip + Exceptions ---------- -Since the result of a ``Future`` is created concurrently to the rest of the program, exceptions must be handled differently. It doesn't matter if an ``UntypedActor`` or the dispatcher is completing the ``Future``, if an ``Exception`` is caught the ``Future`` will contain it instead of a valid result. If a ``Future`` does contain an ``Exception``, calling ``Await.result`` will cause it to be thrown again so it can be handled properly. \ No newline at end of file +Since the result of a ``Future`` is created concurrently to the rest of the program, exceptions must be handled differently. +It doesn't matter if an ``UntypedActor`` or the dispatcher is completing the ``Future``, if an ``Exception`` is caught +the ``Future`` will contain it instead of a valid result. If a ``Future`` does contain an ``Exception``, +calling ``Await.result`` will cause it to be thrown again so it can be handled properly. \ No newline at end of file From e4cd3c327777081fdbbf6bd4f7e670aceaafb2d4 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 24 Jan 2012 17:16:11 +0100 Subject: [PATCH 133/152] Minor adjustments based on feedback. See #1574 --- .../docs/actor/FaultHandlingDocSample.scala | 21 +++++++++++-------- akka-docs/scala/fault-tolerance.rst | 2 +- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSample.scala b/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSample.scala index cb400629d6..49fd40cefb 100644 --- a/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSample.scala +++ b/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSample.scala @@ -31,6 +31,7 @@ object FaultHandlingDocSample extends App { val system = ActorSystem("FaultToleranceSample", config) val worker = system.actorOf(Props[Worker], name = "worker") + // Create an Actor that start the work and listens to progress system.actorOf(Props(new Actor with ActorLogging { // If we don't get any progress within 15 seconds then the service is unavailable context.setReceiveTimeout(15 seconds) @@ -76,12 +77,12 @@ class Worker extends Actor with ActorLogging { } // The sender of the initial Start message will continuously be notified about progress - var progressListener: ActorRef = _ + var progressListener: Option[ActorRef] = None val counterService = context.actorOf(Props[CounterService], name = "counter") def receive = LoggingReceive(this) { - case Start if progressListener eq null ⇒ - progressListener = sender + case Start if progressListener.isEmpty ⇒ + progressListener = Some(sender) context.system.scheduler.schedule(Duration.Zero, 1 second, self, Do) case Do ⇒ @@ -90,7 +91,7 @@ class Worker extends Actor with ActorLogging { counterService ! Increment(1) // Send current count to the initial sender - (counterService ? GetCurrentCount).pipeTo(progressListener) + counterService ? GetCurrentCount pipeTo progressListener.get } } @@ -130,16 +131,18 @@ class CounterService extends Actor { initStorage() } + /** + * The child storage is restarted in case of failure, but after 3 restarts, + * and still failing it will be stopped. Better to back-off than continuously + * failing. When it has been stopped we will schedule a Reconnect after a delay. + * Watch the child so we receive Terminated message when it has been terminated. + */ def initStorage() { - // The child storage is restarted in case of failure, but after 3 restarts, - // and still failing it will be stopped. Better to back-off than continuously - // failing. When it has been stopped we will schedule a Reconnect after a delay. - // Watch the child so we receive Terminated message when it has been terminated. storage = Some(context.watch(context.actorOf(Props[Storage], name = "storage"))) // Tell the counter, if any, to use the new storage counter foreach { _ ! UseStorage(storage) } // We need the initial value to be able to operate - storage foreach { _ ! Get(key) } + storage.get ! Get(key) } def receive = LoggingReceive(this) { diff --git a/akka-docs/scala/fault-tolerance.rst b/akka-docs/scala/fault-tolerance.rst index 4edada1a73..1872f7af65 100644 --- a/akka-docs/scala/fault-tolerance.rst +++ b/akka-docs/scala/fault-tolerance.rst @@ -59,7 +59,7 @@ Default Supervisor Strategy When the supervisor strategy is not defined for an actor the following exceptions are handled by default:: - OneForOneStrategy { + OneForOneStrategy() { case _: ActorInitializationException ⇒ Stop case _: ActorKilledException ⇒ Stop case _: Exception ⇒ Restart From 96e072602e7f694b0609666af441a88ec5d754e4 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 24 Jan 2012 17:44:01 +0100 Subject: [PATCH 134/152] DOC: Typo --- .../scala/code/akka/docs/actor/FaultHandlingDocSample.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSample.scala b/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSample.scala index 49fd40cefb..1b013cd50e 100644 --- a/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSample.scala +++ b/akka-docs/scala/code/akka/docs/actor/FaultHandlingDocSample.scala @@ -157,9 +157,9 @@ class CounterService extends Actor { backlog foreach { c ! _ } backlog = IndexedSeq.empty - case msg @ Increment(n) ⇒ forwardOrPlaceInBacklow(msg) + case msg @ Increment(n) ⇒ forwardOrPlaceInBacklog(msg) - case msg @ GetCurrentCount ⇒ forwardOrPlaceInBacklow(msg) + case msg @ GetCurrentCount ⇒ forwardOrPlaceInBacklog(msg) case Terminated(actorRef) if Some(actorRef) == storage ⇒ // After 3 restarts the storage child is stopped. @@ -175,7 +175,7 @@ class CounterService extends Actor { initStorage() } - def forwardOrPlaceInBacklow(msg: Any) { + def forwardOrPlaceInBacklog(msg: Any) { // We need the initial value from storage before we can start delegate to the counter. // Before that we place the messages in a backlog, to be sent to the counter when // it is initialized. From 6265f68ba9bfd69be7c12d17e8f14d7051603be2 Mon Sep 17 00:00:00 2001 From: Patrik Nordwall Date: Tue, 24 Jan 2012 20:58:32 +0100 Subject: [PATCH 135/152] Fixed conflicting dependencies. See #1718 * Duplicate jar files, of different versions were included in dist * In mongoMailbox I forced netty-3.2.6.Final -> netty-3.3.0.Final, different organization also * In mongoMailbox I forced commons-pool-1.5.5 -> commons-pool-1.5.6 * In mongoMailbox I forced slf4j-api-1.6.1 -> slf4j-api-1.6.4 * In redisMailbox I forced slf4j-api-1.6.1 -> slf4j-api-1.6.4 --- project/AkkaBuild.scala | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index f32987fb45..d104a9db5d 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -190,6 +190,7 @@ object AkkaBuild extends Build { dependencies = Seq(mailboxesCommon % "compile;test->test"), settings = defaultSettings ++ Seq( libraryDependencies ++= Dependencies.mongoMailbox, + ivyXML := Dependencies.mongoMailboxExcludes, testMongoMailbox := false, testOptions in Test <+= testMongoMailbox map { test => Tests.Filter(s => test) } ) @@ -428,9 +429,20 @@ object Dependencies { val beanstalkMailbox = Seq(beanstalk, Test.junit) - val redisMailbox = Seq(redis, Test.junit) + val redisMailbox = Seq(slf4jApi, redis, Test.junit) - val mongoMailbox = Seq(mongoAsync, twttrUtilCore, Test.junit) + val mongoMailbox = Seq(slf4jApi, commonsPool, mongoAsync, twttrUtilCore, Test.junit) + + val mongoMailboxExcludes = { + + + + + + + + + } val zookeeperMailbox = Seq(zkClient, zookeeper, Test.junit) @@ -478,6 +490,7 @@ object Dependency { val camelSpring = "org.apache.camel" % "camel-spring" % V.Camel // ApacheV2 val commonsCodec = "commons-codec" % "commons-codec" % "1.4" // ApacheV2 val commonsIo = "commons-io" % "commons-io" % "2.0.1" // ApacheV2 + val commonsPool = "commons-pool" % "commons-pool" % "1.5.6" // ApacheV2 val guice = "org.guiceyfruit" % "guice-all" % "2.0" // ApacheV2 val h2Lzf = "voldemort.store.compress" % "h2-lzf" % "1.0" // ApacheV2 val jacksonCore = "org.codehaus.jackson" % "jackson-core-asl" % V.Jackson // ApacheV2 From 9d18b620a9e725a5253f88e2784bd1c381155d1c Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 24 Jan 2012 21:19:47 +0100 Subject: [PATCH 136/152] tighten static type of Duration factories - return FiniteDuration whenever possible to allow statically guarding against infinities - add Ordering[Duration] and Ordering[FiniteDuration] instances - avoid use of structural types in multiplication enrichments - had to fix type inference problem due to too precise type in DurationSpec (view bounds vs. local type inference vs. operator precedence) --- .../test/scala/akka/util/DurationSpec.scala | 9 +++--- .../src/main/scala/akka/util/Duration.scala | 29 +++++++++++++------ .../scala/akka/util/duration/package.scala | 22 +++++++++----- 3 files changed, 40 insertions(+), 20 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/util/DurationSpec.scala b/akka-actor-tests/src/test/scala/akka/util/DurationSpec.scala index dd2867019c..d66103ad03 100644 --- a/akka-actor-tests/src/test/scala/akka/util/DurationSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/util/DurationSpec.scala @@ -53,11 +53,12 @@ class DurationSpec extends WordSpec with MustMatchers { "support fromNow" in { val dead = 2.seconds.fromNow val dead2 = 2 seconds fromNow - dead.timeLeft must be > 1.second - dead2.timeLeft must be > 1.second + // view bounds vs. very local type inference vs. operator precedence: sigh + dead.timeLeft must be > (1 second: Duration) + dead2.timeLeft must be > (1 second: Duration) 1.second.sleep - dead.timeLeft must be < 1.second - dead2.timeLeft must be < 1.second + dead.timeLeft must be < (1 second: Duration) + dead2.timeLeft must be < (1 second: Duration) } } diff --git a/akka-actor/src/main/scala/akka/util/Duration.scala b/akka-actor/src/main/scala/akka/util/Duration.scala index c75fa132ae..00dc1ae7e1 100644 --- a/akka-actor/src/main/scala/akka/util/Duration.scala +++ b/akka-actor/src/main/scala/akka/util/Duration.scala @@ -51,11 +51,11 @@ object Deadline { object Duration { implicit def timeLeft(implicit d: Deadline): Duration = d.timeLeft - def apply(length: Long, unit: TimeUnit): Duration = new FiniteDuration(length, unit) - def apply(length: Double, unit: TimeUnit): Duration = fromNanos(unit.toNanos(1) * length) - def apply(length: Long, unit: String): Duration = new FiniteDuration(length, timeUnit(unit)) + def apply(length: Long, unit: TimeUnit): FiniteDuration = new FiniteDuration(length, unit) + def apply(length: Double, unit: TimeUnit): FiniteDuration = fromNanos(unit.toNanos(1) * length) + def apply(length: Long, unit: String): FiniteDuration = new FiniteDuration(length, timeUnit(unit)) - def fromNanos(nanos: Long): Duration = { + def fromNanos(nanos: Long): FiniteDuration = { if (nanos % 86400000000000L == 0) { Duration(nanos / 86400000000000L, DAYS) } else if (nanos % 3600000000000L == 0) { @@ -73,7 +73,7 @@ object Duration { } } - def fromNanos(nanos: Double): Duration = fromNanos((nanos + 0.5).asInstanceOf[Long]) + def fromNanos(nanos: Double): FiniteDuration = fromNanos((nanos + 0.5).asInstanceOf[Long]) /** * Construct a Duration by parsing a String. In case of a format error, a @@ -132,7 +132,7 @@ object Duration { case "ns" | "nano" | "nanos" | "nanosecond" | "nanoseconds" ⇒ NANOSECONDS } - val Zero: Duration = new FiniteDuration(0, NANOSECONDS) + 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 @@ -205,10 +205,14 @@ object Duration { } // Java Factories - def create(length: Long, unit: TimeUnit): Duration = apply(length, unit) - def create(length: Double, unit: TimeUnit): Duration = apply(length, unit) - def create(length: Long, unit: String): Duration = apply(length, unit) + def create(length: Long, unit: TimeUnit): FiniteDuration = apply(length, unit) + def create(length: Double, unit: TimeUnit): FiniteDuration = apply(length, unit) + def create(length: Long, unit: String): FiniteDuration = apply(length, unit) def parse(s: String): Duration = unapply(s).get + + implicit object DurationIsOrdered extends Ordering[Duration] { + def compare(a: Duration, b: Duration) = a compare b + } } /** @@ -294,6 +298,12 @@ abstract class Duration extends Serializable with Ordered[Duration] { def isFinite() = finite_? } +object FiniteDuration { + implicit object FiniteDurationIsOrdered extends Ordering[FiniteDuration] { + def compare(a: FiniteDuration, b: FiniteDuration) = a compare b + } +} + class FiniteDuration(val length: Long, val unit: TimeUnit) extends Duration { import Duration._ @@ -549,6 +559,7 @@ case class Timeout(duration: Duration) { } object Timeout { + /** * A timeout with zero duration, will cause most requests to always timeout. */ diff --git a/akka-actor/src/main/scala/akka/util/duration/package.scala b/akka-actor/src/main/scala/akka/util/duration/package.scala index b052cca93a..7f14a0be48 100644 --- a/akka-actor/src/main/scala/akka/util/duration/package.scala +++ b/akka-actor/src/main/scala/akka/util/duration/package.scala @@ -9,19 +9,19 @@ import java.util.concurrent.TimeUnit package object duration { trait Classifier[C] { type R - def convert(d: Duration): R + def convert(d: FiniteDuration): R } object span implicit object spanConvert extends Classifier[span.type] { - type R = Duration - def convert(d: Duration) = d + type R = FiniteDuration + def convert(d: FiniteDuration) = d } object fromNow implicit object fromNowConvert extends Classifier[fromNow.type] { type R = Deadline - def convert(d: Duration) = Deadline.now + d + def convert(d: FiniteDuration) = Deadline.now + d } implicit def intToDurationInt(n: Int) = new DurationInt(n) @@ -32,13 +32,21 @@ package object duration { implicit def pairLongToDuration(p: (Long, TimeUnit)) = Duration(p._1, p._2) implicit def durationToPair(d: Duration) = (d.length, d.unit) - implicit def intMult(i: Int) = new { + /* + * avoid reflection based invocation by using non-duck type + */ + class IntMult(i: Int) { def *(d: Duration) = d * i } - implicit def longMult(l: Long) = new { + implicit def intMult(i: Int) = new IntMult(i) + + class LongMult(l: Long) { def *(d: Duration) = d * l } - implicit def doubleMult(f: Double) = new { + implicit def longMult(l: Long) = new LongMult(l) + + class DoubleMult(f: Double) { def *(d: Duration) = d * f } + implicit def doubleMult(f: Double) = new DoubleMult(f) } From 2ac8a25b72db09ed9384f48b4f44695bff814a28 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Tue, 24 Jan 2012 13:37:26 -0700 Subject: [PATCH 137/152] More work on IO reST docs --- .../scala/code/akka/docs/io/HTTPServer.scala | 8 ++ akka-docs/scala/io.rst | 132 +++++++++++------- 2 files changed, 92 insertions(+), 48 deletions(-) diff --git a/akka-docs/scala/code/akka/docs/io/HTTPServer.scala b/akka-docs/scala/code/akka/docs/io/HTTPServer.scala index b47b7ace44..837dbf4264 100644 --- a/akka-docs/scala/code/akka/docs/io/HTTPServer.scala +++ b/akka-docs/scala/code/akka/docs/io/HTTPServer.scala @@ -3,10 +3,13 @@ */ package akka.docs.io +//#imports import akka.actor._ import akka.util.{ ByteString, ByteStringBuilder } import java.net.InetSocketAddress +//#imports +//#actor class HttpServer(port: Int) extends Actor { val state = IO.IterateeRef.Map.async[IO.Handle]()(context.dispatcher) @@ -31,7 +34,9 @@ class HttpServer(port: Int) extends Actor { } } +//#actor +//#actor-companion object HttpServer { import HttpIteratees._ @@ -54,6 +59,7 @@ object HttpServer { } } +//#actor-companion //#request-class case class Request(meth: String, path: List[String], query: Option[String], httpver: String, headers: List[Header], body: Option[ByteString]) @@ -209,8 +215,10 @@ object OKResponse { case class OKResponse(body: ByteString, keepAlive: Boolean) //#ok-response +//#main object Main extends App { val port = Option(System.getenv("PORT")) map (_.toInt) getOrElse 8080 val system = ActorSystem() val server = system.actorOf(Props(new HttpServer(port))) } +//#main diff --git a/akka-docs/scala/io.rst b/akka-docs/scala/io.rst index 81c4eaf2da..dad57c6b66 100644 --- a/akka-docs/scala/io.rst +++ b/akka-docs/scala/io.rst @@ -10,7 +10,7 @@ IO (Scala) Introduction ------------ -This documentation is in progress. More to come. +This documentation is in progress and some sections may be incomplete. More will be coming. Components ---------- @@ -22,29 +22,7 @@ A primary goal of Akka's IO module is to only communicate between actors with im ``ByteString`` is a `Rope-like `_ data structure that is immutable and efficient. When 2 ``ByteString``\s are concatenated together they are both stored within the resulting ``ByteString`` instead of copying both to a new ``Array``. Operations such as ``drop`` and ``take`` return ``ByteString``\s that still reference the original ``Array``, but just change the offset and length that is visible. Great care has also been taken to make sure that the internal ``Array`` cannot be modified. Whenever a potentially unsafe ``Array`` is used to create a new ``ByteString`` a defensive copy is created. -``ByteString`` inherits all methods from ``IndexedSeq``, and it also has some new ones: - -copyToBuffer(buffer: ByteBuffer): Int - Copy as many bytes as possible to a ``ByteBuffer``, starting from it's current position. This method will not overflow the buffer. It returns the number of bytes copied. - -compact: ByteString - Creates a new ``ByteString`` with all contents compacted into a single byte array. If the contents of this ``ByteString`` are already compacted it will return itself unchanged. - -asByteBuffer: ByteBuffer - If possible this will return a read-only ``ByteBuffer`` that wraps the internal byte array. If this ``ByteString`` contains more then one byte array then this method will return the result of ``toByteBuffer``. - -toByteBuffer: ByteBuffer - Creates a new ByteBuffer with a copy of all bytes contained in this ``ByteString``. - -decodeString(charset: String): String - Decodes this ``ByteString`` using a charset to produce a ``String``. - -utf8String: String - Decodes this ``ByteString`` as a UTF-8 encoded String. - -There are also several factory methods in the ``ByteString`` companion object to assist in creating a new ``ByteString``. The ``apply`` method accepts ``Array[Byte]``, ``Byte*``, ``ByteBuffer``, ``String``, as well as a ``String`` with a charset. There is also ``fromArray(array, offset, length)`` for creating a ``ByteString`` using only part of an ``Array[Byte]``. - -Finally, there is a ``ByteStringBuilder`` to build up a ``ByteString`` using Scala's mutable ``Builder`` concept. It can be especially useful when many ``ByteString``\s need to be concatenated in a performance critical section of a program. +``ByteString`` inherits all methods from ``IndexedSeq``, and it also has some new ones. For more information, look up the ``akka.util.ByteString`` class and it's companion object in the `ScalaDoc `_. IO.Handle ^^^^^^^^^ @@ -54,44 +32,71 @@ IO.Handle IOManager ^^^^^^^^^ -The ``IOManager`` takes care of the low level IO details. Each ``ActorSystem`` has it's own ``IOManager``, which can be accessed calling ``IOManager(system: ActorSystem)``. ``Actor``\s communicate with the ``IOManager`` with specific messages. The messages sent from an ``Actor`` to the ``IOManager`` are created and sent from certain methods: +The ``IOManager`` takes care of the low level IO details. Each ``ActorSystem`` has it's own ``IOManager``, which can be accessed calling ``IOManager(system: ActorSystem)``. ``Actor``\s communicate with the ``IOManager`` with specific messages. The messages sent from an ``Actor`` to the ``IOManager`` are handled automatically when using certain methods and the messagegs sent from an ``IOManager`` are handled within an ``Actor``\'s ``receive`` method. -IOManager(system).connect(address: SocketAddress): IO.SocketHandle - Opens a ``SocketChannel`` and connects to an address. Can also use ``connect(host: String, port: Int)``. +Connecting to a remote host: -IOManager(system).listen(address: SocketAddress): IO.ServerHandle - Opens a ``ServerSocketChannel`` and listens on an address. Can also use ``listen(host: String, port: Int)``. +.. code-block:: scala -socketHandle.write(bytes: ByteString) - Write to the ``SocketChannel``. + val address = new InetSocketAddress("remotehost", 80) + val socket = IOManager(actorSystem).connect(address) -serverHandle.accept(): IO.SocketHandle - Accepts an incoming connection, and returns the ``IO.SocketHandle`` for the new connection. +.. code-block:: scala -handle.close() - Closes the ``Channel``. + val socket = IOManager(actorSystem).connect("remotehost", 80) -Messages that the ``IOManager`` can send to an ``Actor`` are: +Creating a server: -IO.Listening(server: IO.ServerHandle, address: SocketAddress) - Sent when a ``ServerSocketChannel`` is created. If port 0 (random port) was requested then the address returned here will contain the actual port. +.. code-block:: scala -IO.Connected(socket: IO.SocketHandle, address: SocketAddress) - Sent after a ``SocketChannel`` has successfully connected. + val address = new InetSocketAddress("localhost", 80) + val serverSocket = IOManager(actorSystem).listen(address) -IO.NewClient(server: IO.ServerHandle) - Sent when a new client has connected to a ``ServerSocketChannel``. The ``accept`` method must be called on the ``IO.ServerHandle`` in order to get the ``IO.SocketHandle`` to communicate to the new client. +.. code-block:: scala -IO.Read(handle: IO.ReadHandle, bytes: ByteString) - Sent when bytes have been read from a ``SocketChannel``. The handle is a ``IO.ReadHandle``, which is a superclass of ``IO.SocketHandle``. + val serverSocket = IOManager(actorSystem).listen("localhost", 80) -IO.Closed(handle: IO.Handle, cause: Option[Exception]) - Sent when a ``Channel`` has closed. If an ``Exception`` was thrown due to this ``Channel`` closing it will be contained here. +Receiving messages from the ``IOManager``: + +.. code-block:: scala + + def receive = { + + case IO.Listening(server, address) => + println("The server is listening on socket " + address) + + case IO.Connected(socket, address) => + println("Successfully connected to " + address) + + case IO.NewClient(server) => + println("New incoming connection on server") + val socket = server.accept() + println("Writing to new client socket") + socket.write(bytes) + println("Closing socket") + socket.close() + + case IO.Read(socket, bytes) => + println("Received incoming data from socket") + + case IO.Closed(socket: IO.SocketHandle, cause) => + println("Socket has closed, cause: " + cause) + + case IO.Closed(server: IO.ServerHandle, cause) => + println("Server socket has closed, cause: " + cause) + + } IO.Iteratee ^^^^^^^^^^^ -See example below. +Included with Akka's IO module is a basic implementation of ``Iteratee``\s. ``Iteratee``\s are an effective way of handling a stream of data without needing to wait for all the data to arrive. This is especially useful when dealing with non blocking IO since we will usually receive data in chunks which may not include enough information to process, or it may contain much more data then we currently need. + +This ``Iteratee`` implementation is much more basic then what is usually found. There is only support for ``ByteString`` input, and enumerators aren't used. The reason for this limited implementation is to reduce the amount of explicit type signatures needed and to keep things simple. It is important to note that Akka's ``Iteratee``\s are completely optional, incoming data can be handled in any way, including other ``Iteratee`` libraries. + +``Iteratee``\s work by processing the data that it is given and returning either the result (with any unused input) or a continuation if more input is needed. They are monadic, so methods like ``flatMap`` can be used to pass the result of an ``Iteratee`` to another. + +The basic ``Iteratee``\s included in the IO module can all be found in the `ScalaDoc `_ under ``akka.actor.IO``, and some of them are covered in the example below. Examples -------- @@ -99,6 +104,11 @@ Examples Http Server ^^^^^^^^^^^ +This example will create a simple high performance HTTP server. We begin with our imports: + +.. includecode:: code/akka/docs/io/HTTPServer.scala + :include: imports + Some commonly used constants: .. includecode:: code/akka/docs/io/HTTPServer.scala @@ -132,7 +142,7 @@ Reading the request URI is a bit more complicated because we want to parse the i For this example we are only interested in handling absolute paths. To detect if we the URI is an absolute path we use ``IO.peek(length: Int): Iteratee[ByteString]``, which returns a ``ByteString`` of the request length but doesn't actually consume the input. We peek at the next bit of input and see if it matches our ``PATH`` constant (defined above as ``ByteString("/")``). If it doesn't match we throw an error, but for a more robust solution we would want to handle other valid URIs. -Reading the URI path will be our most complex ``Iteratee``. It involves a recursive method that reads in each path segment of the URI: +Next we handle the path itself: .. includecode:: code/akka/docs/io/HTTPServer.scala :include: read-path @@ -141,15 +151,41 @@ The ``step`` method is a recursive method that takes a ``List`` of the accumulat If after reading in a path segment the next input does not start with a path, we reverse the accumulated segments and return it (dropping the last segment if it is blank). +Following the path we read in the query (if it exists): + .. includecode:: code/akka/docs/io/HTTPServer.scala :include: read-query +It is much simpler then reading the path since we aren't doing any parsing of the query since there is no standard format of the query string. + +Both the path and query used the ``readUriPart`` ``Iteratee``, which is next: + .. includecode:: code/akka/docs/io/HTTPServer.scala :include: read-uri-part +Here we have several ``Set``\s that contain valid characters pulled from the URI spec. The ``readUriPart`` method takes a ``Set`` of valid characters (already mapped to ``Byte``\s) and will continue to match characters until it reaches on that is not part of the ``Set``. If it is a percent encoded character then that is handled as a valid character and processing continues, or else we are done collecting this part of the URI. + +Headers are next: + .. includecode:: code/akka/docs/io/HTTPServer.scala :include: read-headers +And if applicable, we read in the message body: + .. includecode:: code/akka/docs/io/HTTPServer.scala :include: read-body +Finally we get to the actual ``Actor``: + +.. includecode:: code/akka/docs/io/HTTPServer.scala + :include: actor + +And it's companion object: + +.. includecode:: code/akka/docs/io/HTTPServer.scala + :include: actor-companion + +A ``main`` method to start everything up: + +.. includecode:: code/akka/docs/io/HTTPServer.scala + :include: main From f7a45c0e8f58fbe59e0543cec9e80cc260b79af1 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 25 Jan 2012 15:38:04 +0100 Subject: [PATCH 138/152] Removing dead imports from akka-actor --- .../java/akka/dispatch/AbstractMessageDispatcher.java | 1 - akka-actor/src/main/scala/akka/actor/Actor.scala | 11 ----------- akka-actor/src/main/scala/akka/actor/ActorRef.scala | 5 +---- .../src/main/scala/akka/actor/ActorRefProvider.scala | 3 +-- .../src/main/scala/akka/actor/ActorSystem.scala | 2 -- akka-actor/src/main/scala/akka/actor/Deployer.scala | 5 ----- akka-actor/src/main/scala/akka/actor/FSM.scala | 1 - .../src/main/scala/akka/actor/FaultHandling.scala | 1 - akka-actor/src/main/scala/akka/actor/Locker.scala | 1 - akka-actor/src/main/scala/akka/actor/Props.scala | 1 - akka-actor/src/main/scala/akka/actor/Scheduler.scala | 2 +- akka-actor/src/main/scala/akka/actor/TypedActor.scala | 4 ++-- .../src/main/scala/akka/actor/UntypedActor.scala | 3 +-- .../scala/akka/dispatch/BalancingDispatcher.scala | 6 +----- .../src/main/scala/akka/dispatch/Dispatchers.scala | 3 --- akka-actor/src/main/scala/akka/dispatch/Future.scala | 11 ++++------- akka-actor/src/main/scala/akka/dispatch/Mailbox.scala | 1 - .../main/scala/akka/dispatch/PinnedDispatcher.scala | 5 ----- .../src/main/scala/akka/dispatch/PromiseStream.scala | 2 +- .../main/scala/akka/dispatch/ThreadPoolBuilder.scala | 2 +- .../src/main/scala/akka/dispatch/japi/Future.scala | 3 +-- .../main/scala/akka/routing/ConnectionManager.scala | 5 ----- akka-actor/src/main/scala/akka/routing/Routing.scala | 6 ++---- akka-actor/src/main/scala/akka/util/ByteString.scala | 6 +++--- akka-actor/src/main/scala/akka/util/Duration.scala | 3 +-- akka-actor/src/main/scala/akka/util/Helpers.scala | 1 - .../src/main/scala/akka/util/ReflectiveAccess.scala | 1 - 27 files changed, 20 insertions(+), 75 deletions(-) diff --git a/akka-actor/src/main/java/akka/dispatch/AbstractMessageDispatcher.java b/akka-actor/src/main/java/akka/dispatch/AbstractMessageDispatcher.java index 8248fd9e07..6720cf2b54 100644 --- a/akka-actor/src/main/java/akka/dispatch/AbstractMessageDispatcher.java +++ b/akka-actor/src/main/java/akka/dispatch/AbstractMessageDispatcher.java @@ -6,7 +6,6 @@ package akka.dispatch; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.concurrent.atomic.AtomicLongFieldUpdater; -import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; abstract class AbstractMessageDispatcher { private volatile int _shutdownSchedule; // not initialized because this is faster: 0 == UNSCHEDULED diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index 7a1f640160..94aef4bdef 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -4,20 +4,9 @@ package akka.actor -import akka.dispatch._ -import akka.routing._ -import akka.util.Duration -import akka.japi.{ Creator, Procedure } -import akka.serialization.{ Serializer, Serialization } -import akka.event.Logging.Debug -import akka.event.LogSource -import akka.experimental import akka.AkkaException import scala.reflect.BeanProperty import scala.util.control.NoStackTrace -import com.eaio.uuid.UUID -import java.util.concurrent.TimeUnit -import java.util.{ Collection ⇒ JCollection } import java.util.regex.Pattern /** diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 5ccce6906c..fa6c9962e7 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -9,13 +9,10 @@ import akka.util._ import scala.collection.immutable.Stack import java.lang.{ UnsupportedOperationException, IllegalStateException } import akka.serialization.Serialization -import java.util.concurrent.TimeUnit import akka.event.EventStream -import akka.event.DeathWatch import scala.annotation.tailrec -import java.util.concurrent.{ ConcurrentHashMap, TimeoutException } +import java.util.concurrent.{ ConcurrentHashMap } import akka.event.LoggingAdapter -import java.util.concurrent.atomic.AtomicBoolean /** * Immutable and serializable handle to an actor, which may or may not reside diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 2ac45e90a9..a7e9a2163e 100755 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -5,11 +5,10 @@ package akka.actor import java.util.concurrent.atomic.AtomicLong -import akka.config.ConfigurationException import akka.dispatch._ import akka.routing._ import akka.AkkaException -import akka.util.{ Duration, Switch, Helpers, Timeout } +import akka.util.{ Switch, Helpers } import akka.event._ /** diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index c6842d0dc2..fc4a6f1f19 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -7,8 +7,6 @@ import akka.config.ConfigurationException import akka.event._ import akka.dispatch._ import akka.pattern.ask -import akka.util.duration._ -import akka.util.Timeout._ import org.jboss.netty.akka.util.HashedWheelTimer import java.util.concurrent.TimeUnit.MILLISECONDS import com.typesafe.config.Config diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index a13d3a8424..c8e780d5c2 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -4,12 +4,7 @@ package akka.actor -import collection.immutable.Seq -import akka.event.Logging -import akka.AkkaException -import akka.config.ConfigurationException import akka.util.Duration -import akka.event.EventStream import com.typesafe.config._ import akka.routing._ import java.util.concurrent.{ TimeUnit, ConcurrentHashMap } diff --git a/akka-actor/src/main/scala/akka/actor/FSM.scala b/akka-actor/src/main/scala/akka/actor/FSM.scala index eb7081899c..5660811c00 100644 --- a/akka-actor/src/main/scala/akka/actor/FSM.scala +++ b/akka-actor/src/main/scala/akka/actor/FSM.scala @@ -7,7 +7,6 @@ import akka.util._ import scala.collection.mutable import akka.event.Logging -import akka.util.Duration._ import akka.routing.{ Deafen, Listen, Listeners } object FSM { diff --git a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala index 8d26567c37..71102ffb03 100644 --- a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala @@ -4,7 +4,6 @@ package akka.actor import java.util.concurrent.TimeUnit -import scala.annotation.tailrec import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import java.lang.{ Iterable ⇒ JIterable } diff --git a/akka-actor/src/main/scala/akka/actor/Locker.scala b/akka-actor/src/main/scala/akka/actor/Locker.scala index 9e34f02332..4f9caaeedc 100644 --- a/akka-actor/src/main/scala/akka/actor/Locker.scala +++ b/akka-actor/src/main/scala/akka/actor/Locker.scala @@ -5,7 +5,6 @@ package akka.actor import akka.dispatch._ import akka.util.Duration -import akka.util.duration._ import java.util.concurrent.ConcurrentHashMap import akka.event.DeathWatch diff --git a/akka-actor/src/main/scala/akka/actor/Props.scala b/akka-actor/src/main/scala/akka/actor/Props.scala index c0b84d0017..cd9a62abe7 100644 --- a/akka-actor/src/main/scala/akka/actor/Props.scala +++ b/akka-actor/src/main/scala/akka/actor/Props.scala @@ -6,7 +6,6 @@ package akka.actor import akka.dispatch._ import akka.japi.Creator -import akka.util._ import collection.immutable.Stack import akka.routing._ diff --git a/akka-actor/src/main/scala/akka/actor/Scheduler.scala b/akka-actor/src/main/scala/akka/actor/Scheduler.scala index a2451173b3..72d429b450 100644 --- a/akka-actor/src/main/scala/akka/actor/Scheduler.scala +++ b/akka-actor/src/main/scala/akka/actor/Scheduler.scala @@ -13,7 +13,7 @@ package akka.actor import akka.util.Duration -import org.jboss.netty.akka.util.{ Timer, TimerTask, HashedWheelTimer, Timeout ⇒ HWTimeout } +import org.jboss.netty.akka.util.{ TimerTask, HashedWheelTimer, Timeout ⇒ HWTimeout } import akka.event.LoggingAdapter import akka.dispatch.MessageDispatcher import java.io.Closeable diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index ce487ec2ef..69c8e44fc3 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -6,9 +6,9 @@ package akka.actor import akka.japi.{ Creator, Option ⇒ JOption } import java.lang.reflect.{ InvocationTargetException, Method, InvocationHandler, Proxy } -import akka.util.{ Duration, Timeout } +import akka.util.{ Timeout } import java.util.concurrent.atomic.{ AtomicReference ⇒ AtomVar } -import akka.serialization.{ Serializer, Serialization, SerializationExtension } +import akka.serialization.{ Serialization, SerializationExtension } import akka.dispatch._ import java.util.concurrent.TimeoutException import java.lang.IllegalStateException diff --git a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala index d06896a10d..6dd4d8c2c5 100644 --- a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala @@ -4,8 +4,7 @@ package akka.actor -import akka.japi.{ Creator, Procedure } -import akka.dispatch.{ MessageDispatcher, Promise } +import akka.japi.{ Creator } /** * Actor base trait that should be extended by or mixed to create an Actor with the semantics of the 'Actor Model': diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index cc2d7a5e96..c4742df81a 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -5,13 +5,9 @@ package akka.dispatch import util.DynamicVariable -import akka.actor.{ ActorCell, Actor, IllegalActorStateException, ActorRef } +import akka.actor.{ ActorCell, ActorRef } import java.util.concurrent.{ LinkedBlockingQueue, ConcurrentLinkedQueue, ConcurrentSkipListSet } -import java.util.{ Comparator, Queue } import annotation.tailrec -import akka.actor.ActorSystem -import akka.event.EventStream -import akka.actor.Scheduler import java.util.concurrent.atomic.AtomicBoolean import akka.util.Duration diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala index b622c52c74..3871905905 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala @@ -9,12 +9,9 @@ import akka.util.{ Duration, ReflectiveAccess } import akka.actor.ActorSystem import akka.event.EventStream import akka.actor.Scheduler -import akka.actor.ActorSystem.Settings import com.typesafe.config.Config import com.typesafe.config.ConfigFactory -import akka.config.ConfigurationException import akka.event.Logging.Warning -import akka.actor.Props import java.util.concurrent.{ ThreadFactory, TimeUnit, ConcurrentHashMap } trait DispatcherPrerequisites { diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 09ce22d6b8..2663aff23e 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -4,25 +4,22 @@ package akka.dispatch -import akka.AkkaException import akka.event.Logging.Error -import akka.util.Timeout import scala.Option -import akka.japi.{ Procedure, Function ⇒ JFunc, Option ⇒ JOption } +import akka.japi.{ Function ⇒ JFunc, Option ⇒ JOption } import scala.util.continuations._ -import java.util.concurrent.TimeUnit.{ NANOSECONDS, MILLISECONDS } +import java.util.concurrent.TimeUnit.{ NANOSECONDS } import java.lang.{ Iterable ⇒ JIterable } import java.util.{ LinkedList ⇒ JLinkedList } import scala.annotation.tailrec import scala.collection.mutable.Stack -import akka.util.{ Switch, Duration, BoxedType } -import java.util.concurrent.atomic.{ AtomicReferenceFieldUpdater, AtomicInteger, AtomicBoolean } +import akka.util.{ Duration, BoxedType } +import java.util.concurrent.atomic.{ AtomicReferenceFieldUpdater, AtomicInteger } import akka.dispatch.Await.CanAwait import java.util.concurrent._ -import akka.actor.ActorSystem object Await { sealed trait CanAwait diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index 40d4650f0d..d2bc7ff01d 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -10,7 +10,6 @@ import akka.actor.{ ActorCell, ActorRef } import java.util.concurrent._ import annotation.tailrec import akka.event.Logging.Error -import com.typesafe.config.Config import akka.actor.ActorContext class MessageQueueAppendFailedException(message: String, cause: Throwable = null) extends AkkaException(message, cause) diff --git a/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala index c88113fc64..246b1f591a 100644 --- a/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala @@ -4,13 +4,8 @@ package akka.dispatch -import java.util.concurrent.atomic.AtomicReference import akka.actor.ActorCell -import akka.actor.ActorSystem -import akka.event.EventStream -import akka.actor.Scheduler import akka.util.Duration -import java.util.concurrent.TimeUnit /** * Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue. diff --git a/akka-actor/src/main/scala/akka/dispatch/PromiseStream.scala b/akka-actor/src/main/scala/akka/dispatch/PromiseStream.scala index b6305701b2..b1c25f55e1 100644 --- a/akka-actor/src/main/scala/akka/dispatch/PromiseStream.scala +++ b/akka-actor/src/main/scala/akka/dispatch/PromiseStream.scala @@ -6,7 +6,7 @@ package akka.dispatch import java.util.concurrent.atomic.AtomicReference import scala.util.continuations._ -import scala.annotation.{ tailrec } +import scala.annotation.tailrec import akka.util.Timeout object PromiseStream { diff --git a/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala index 9601c4dd5e..f8927f667a 100644 --- a/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala +++ b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala @@ -5,7 +5,7 @@ package akka.dispatch import java.util.Collection -import java.util.concurrent.atomic.{ AtomicLong, AtomicInteger } +import java.util.concurrent.atomic.AtomicLong import akka.util.Duration import java.util.concurrent._ diff --git a/akka-actor/src/main/scala/akka/dispatch/japi/Future.scala b/akka-actor/src/main/scala/akka/dispatch/japi/Future.scala index ac4ef7694e..a237c0c647 100644 --- a/akka-actor/src/main/scala/akka/dispatch/japi/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/japi/Future.scala @@ -3,8 +3,7 @@ */ package akka.dispatch.japi -import akka.util.Timeout -import akka.japi.{ Procedure2, Procedure, Function ⇒ JFunc, Option ⇒ JOption } +import akka.japi.{ Procedure2, Procedure, Function ⇒ JFunc } /* Java API */ trait Future[+T] { self: akka.dispatch.Future[T] ⇒ diff --git a/akka-actor/src/main/scala/akka/routing/ConnectionManager.scala b/akka-actor/src/main/scala/akka/routing/ConnectionManager.scala index 3ce73f5020..94ceffb91f 100644 --- a/akka-actor/src/main/scala/akka/routing/ConnectionManager.scala +++ b/akka-actor/src/main/scala/akka/routing/ConnectionManager.scala @@ -6,11 +6,6 @@ package akka.routing import akka.actor._ -import scala.annotation.tailrec - -import java.util.concurrent.atomic.{ AtomicReference, AtomicInteger } -import collection.JavaConverters - /** * An Iterable that also contains a version. */ diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index c6fb245d6a..4ef7cff330 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -4,11 +4,9 @@ package akka.routing import akka.actor._ -import akka.dispatch.{ Future, Promise } -import java.util.concurrent.atomic.AtomicLong -import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.atomic.{ AtomicLong, AtomicBoolean } import java.util.concurrent.TimeUnit -import akka.util.{ Duration, Timeout } +import akka.util.Duration import akka.util.duration._ import com.typesafe.config.Config import akka.config.ConfigurationException diff --git a/akka-actor/src/main/scala/akka/util/ByteString.scala b/akka-actor/src/main/scala/akka/util/ByteString.scala index 125b605a89..dd38fe3bdd 100644 --- a/akka-actor/src/main/scala/akka/util/ByteString.scala +++ b/akka-actor/src/main/scala/akka/util/ByteString.scala @@ -3,10 +3,10 @@ package akka.util import java.nio.ByteBuffer import java.nio.charset.Charset -import scala.collection.{ IndexedSeqOptimized, LinearSeq } -import scala.collection.mutable.{ Builder, ArrayBuilder, WrappedArray } +import scala.collection.{ IndexedSeqOptimized } +import scala.collection.mutable.{ Builder, WrappedArray } import scala.collection.immutable.{ IndexedSeq, VectorBuilder } -import scala.collection.generic.{ CanBuildFrom, GenericCompanion } +import scala.collection.generic.{ CanBuildFrom } object ByteString { diff --git a/akka-actor/src/main/scala/akka/util/Duration.scala b/akka-actor/src/main/scala/akka/util/Duration.scala index 00dc1ae7e1..65d6e6148c 100644 --- a/akka-actor/src/main/scala/akka/util/Duration.scala +++ b/akka-actor/src/main/scala/akka/util/Duration.scala @@ -6,8 +6,7 @@ package akka.util import java.util.concurrent.TimeUnit import TimeUnit._ -import java.lang.{ Long ⇒ JLong, Double ⇒ JDouble } -import akka.actor.ActorSystem +import java.lang.{ Double ⇒ JDouble } class TimerException(message: String) extends RuntimeException(message) diff --git a/akka-actor/src/main/scala/akka/util/Helpers.scala b/akka-actor/src/main/scala/akka/util/Helpers.scala index d98e0a49f1..60e6be8b65 100644 --- a/akka-actor/src/main/scala/akka/util/Helpers.scala +++ b/akka-actor/src/main/scala/akka/util/Helpers.scala @@ -3,7 +3,6 @@ */ package akka.util -import java.io.{ PrintWriter, StringWriter } import java.util.Comparator import scala.annotation.tailrec import java.util.regex.Pattern diff --git a/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala b/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala index 18ed6a5020..7c00d69225 100644 --- a/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala +++ b/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala @@ -4,7 +4,6 @@ package akka.util -import akka.actor._ import java.lang.reflect.InvocationTargetException object ReflectiveAccess { From a2c558838d65f26d6a99707cc57dbba68570905d Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 25 Jan 2012 15:42:29 +0100 Subject: [PATCH 139/152] Removing dead imports in akka-migration --- .../src/main/scala/akka/actor/GlobalActorSystem.scala | 1 - akka-actor-migration/src/main/scala/akka/actor/OldActor.scala | 1 - .../src/main/scala/akka/dispatch/OldFuture.scala | 1 - .../src/main/scala/akka/migration/package.scala | 2 +- 4 files changed, 1 insertion(+), 4 deletions(-) diff --git a/akka-actor-migration/src/main/scala/akka/actor/GlobalActorSystem.scala b/akka-actor-migration/src/main/scala/akka/actor/GlobalActorSystem.scala index e0ba90b1f9..694dd5d547 100644 --- a/akka-actor-migration/src/main/scala/akka/actor/GlobalActorSystem.scala +++ b/akka-actor-migration/src/main/scala/akka/actor/GlobalActorSystem.scala @@ -8,7 +8,6 @@ import java.io.File import com.typesafe.config.Config import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigParseOptions -import com.typesafe.config.ConfigResolveOptions @deprecated("use ActorSystem instead", "2.0") object GlobalActorSystem extends ActorSystemImpl("GlobalSystem", OldConfigurationLoader.defaultConfig) { diff --git a/akka-actor-migration/src/main/scala/akka/actor/OldActor.scala b/akka-actor-migration/src/main/scala/akka/actor/OldActor.scala index e923773bd8..49035093dd 100644 --- a/akka-actor-migration/src/main/scala/akka/actor/OldActor.scala +++ b/akka-actor-migration/src/main/scala/akka/actor/OldActor.scala @@ -9,7 +9,6 @@ import akka.dispatch.Future import akka.dispatch.OldFuture import akka.util.Duration import java.util.concurrent.TimeUnit -import java.net.InetSocketAddress import akka.migration.AskableActorRef /** diff --git a/akka-actor-migration/src/main/scala/akka/dispatch/OldFuture.scala b/akka-actor-migration/src/main/scala/akka/dispatch/OldFuture.scala index f53a3dd11b..89941cd066 100644 --- a/akka-actor-migration/src/main/scala/akka/dispatch/OldFuture.scala +++ b/akka-actor-migration/src/main/scala/akka/dispatch/OldFuture.scala @@ -4,7 +4,6 @@ package akka.dispatch import java.util.concurrent.TimeoutException -import akka.util.duration._ import akka.AkkaException import akka.util.BoxedType import akka.util.Duration diff --git a/akka-actor-migration/src/main/scala/akka/migration/package.scala b/akka-actor-migration/src/main/scala/akka/migration/package.scala index 4fd8d5eeea..6b8cad6fe2 100644 --- a/akka-actor-migration/src/main/scala/akka/migration/package.scala +++ b/akka-actor-migration/src/main/scala/akka/migration/package.scala @@ -21,7 +21,7 @@ package object migration { implicit def actorRef2OldActorRef(actorRef: ActorRef) = new OldActorRef(actorRef) class OldActorRef(actorRef: ActorRef) { - @deprecated("Actors are automatically started when creatd, i.e. remove old call to start()", "2.0") + @deprecated("Actors are automatically started when created, i.e. remove old call to start()", "2.0") def start(): ActorRef = actorRef @deprecated("Stop with ActorSystem or ActorContext instead", "2.0") From 08967d414906291f580cc6b961eee12eefc2d71a Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 25 Jan 2012 15:46:45 +0100 Subject: [PATCH 140/152] Removing dead imports in durable mailboxes --- .../akka/actor/mailbox/BeanstalkBasedMailbox.scala | 2 -- .../main/scala/akka/actor/mailbox/DurableMailbox.scala | 10 +++------- .../scala/akka/actor/mailbox/BSONSerialization.scala | 3 +-- .../scala/akka/actor/mailbox/MongoBasedMailbox.scala | 2 +- .../akka/actor/mailbox/ZooKeeperBasedMailbox.scala | 3 --- 5 files changed, 5 insertions(+), 15 deletions(-) diff --git a/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala b/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala index 6715679eb5..cd2ca2d0f0 100644 --- a/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-beanstalk-mailbox/src/main/scala/akka/actor/mailbox/BeanstalkBasedMailbox.scala @@ -5,8 +5,6 @@ package akka.actor.mailbox import com.surftools.BeanstalkClient._ import com.surftools.BeanstalkClientImpl._ -import java.util.concurrent.TimeUnit.MILLISECONDS -import akka.util.Duration import akka.AkkaException import akka.actor.ActorContext import akka.dispatch.Envelope diff --git a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala index 7156a859c1..e00520d92c 100644 --- a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala +++ b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala @@ -3,14 +3,10 @@ */ package akka.actor.mailbox -import akka.actor.ActorContext -import akka.actor.ActorRef -import akka.dispatch.Envelope -import akka.dispatch.DefaultSystemMessageQueue -import akka.dispatch.CustomMailbox +import akka.actor.{ ActorContext, ActorRef } +import akka.dispatch.{ Envelope, DefaultSystemMessageQueue, CustomMailbox } import akka.remote.MessageSerializer -import akka.remote.RemoteProtocol.ActorRefProtocol -import akka.remote.RemoteProtocol.RemoteMessageProtocol +import akka.remote.RemoteProtocol.{ ActorRefProtocol, RemoteMessageProtocol } private[akka] object DurableExecutableMailboxConfig { val Name = "[\\.\\/\\$\\s]".r diff --git a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala index 7da9dd7a3e..470993fdf3 100644 --- a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala +++ b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/BSONSerialization.scala @@ -14,10 +14,9 @@ import org.bson.BSONSerializer import org.bson.DefaultBSONDeserializer import org.bson.DefaultBSONSerializer -import akka.actor.SerializedActorRef import akka.remote.RemoteProtocol.MessageProtocol import akka.remote.MessageSerializer -import akka.actor.{ ActorSystem, ActorSystemImpl, Props } +import akka.actor.{ ActorSystem, ActorSystemImpl } class BSONSerializableMailbox(system: ActorSystem) extends SerializableBSONObject[MongoDurableMessage] with Logging { diff --git a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala index 4eb36eefe1..fed643c7d1 100644 --- a/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-mongo-mailbox/src/main/scala/akka/actor/mailbox/MongoBasedMailbox.scala @@ -10,7 +10,7 @@ import org.bson.collection._ import akka.actor.ActorContext import akka.event.Logging import akka.actor.ActorRef -import akka.dispatch.{ Await, Promise, Envelope, DefaultPromise } +import akka.dispatch.{ Await, Promise, Envelope } import java.util.concurrent.TimeoutException import akka.dispatch.MailboxType import com.typesafe.config.Config diff --git a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala index 4ac254cdc1..1420a8a543 100644 --- a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala +++ b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/main/scala/akka/actor/mailbox/ZooKeeperBasedMailbox.scala @@ -3,10 +3,7 @@ */ package akka.actor.mailbox -import java.util.concurrent.TimeUnit.MILLISECONDS -import akka.util.Duration import akka.AkkaException -import org.I0Itec.zkclient.serialize._ import akka.actor.ActorContext import akka.cluster.zookeeper.AkkaZkClient import akka.dispatch.Envelope From 7f0c4e3d63024030f49b4903a7e0d0671cc64be2 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 25 Jan 2012 15:48:30 +0100 Subject: [PATCH 141/152] Removing dead imports from kernel --- akka-kernel/src/main/scala/akka/kernel/Main.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-kernel/src/main/scala/akka/kernel/Main.scala b/akka-kernel/src/main/scala/akka/kernel/Main.scala index 3b89cf4ec0..ead2c28121 100644 --- a/akka-kernel/src/main/scala/akka/kernel/Main.scala +++ b/akka-kernel/src/main/scala/akka/kernel/Main.scala @@ -7,7 +7,7 @@ package akka.kernel import akka.actor.ActorSystem import java.io.File import java.lang.Boolean.getBoolean -import java.net.{ URL, URLClassLoader } +import java.net.URLClassLoader import java.util.jar.JarFile import scala.collection.JavaConverters._ From 1590438b0c1ff38f8af09577817fd33cbaf20fe7 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 25 Jan 2012 15:55:27 +0100 Subject: [PATCH 142/152] Removing dead imports from akka-remote --- .../main/scala/akka/remote/AccrualFailureDetector.scala | 2 -- akka-remote/src/main/scala/akka/remote/Gossiper.scala | 2 -- .../src/main/scala/akka/remote/NetworkEventStream.scala | 4 +--- akka-remote/src/main/scala/akka/remote/Remote.scala | 4 +--- .../main/scala/akka/remote/RemoteActorRefProvider.scala | 7 ------- .../main/scala/akka/remote/RemoteConnectionManager.scala | 1 - .../src/main/scala/akka/remote/RemoteInterface.scala | 3 --- .../src/main/scala/akka/remote/RemoteSettings.scala | 1 - .../main/scala/akka/remote/netty/NettyRemoteSupport.scala | 3 +-- .../akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala | 3 --- .../scala/akka/remote/NewRemoteActorMultiJvmSpec.scala | 4 ---- .../akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala | 2 -- .../remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala | 2 -- .../ScatterGatherRoutedRemoteActorMultiJvmSpec.scala | 1 - 14 files changed, 3 insertions(+), 36 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala b/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala index 0e653819ee..af74c82040 100644 --- a/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala +++ b/akka-remote/src/main/scala/akka/remote/AccrualFailureDetector.scala @@ -11,8 +11,6 @@ import scala.annotation.tailrec import System.{ currentTimeMillis ⇒ newTimestamp } -import akka.actor.ActorSystem - /** * Implementation of 'The Phi Accrual Failure Detector' by Hayashibara et al. as defined in their paper: * [http://ddg.jaist.ac.jp/pub/HDY+04.pdf] diff --git a/akka-remote/src/main/scala/akka/remote/Gossiper.scala b/akka-remote/src/main/scala/akka/remote/Gossiper.scala index 2ff25df939..6cdea287d4 100644 --- a/akka-remote/src/main/scala/akka/remote/Gossiper.scala +++ b/akka-remote/src/main/scala/akka/remote/Gossiper.scala @@ -9,7 +9,6 @@ import akka.actor.Status._ import akka.event.Logging import akka.util.Duration import akka.config.ConfigurationException -import akka.serialization.SerializationExtension import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.TimeUnit.SECONDS @@ -19,7 +18,6 @@ import System.{ currentTimeMillis ⇒ newTimestamp } import scala.collection.immutable.Map import scala.annotation.tailrec -import com.google.protobuf.ByteString import java.util.concurrent.TimeoutException import akka.dispatch.Await import akka.pattern.ask diff --git a/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala b/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala index c108b54ae7..4fb970262a 100644 --- a/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala +++ b/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala @@ -5,9 +5,7 @@ package akka.remote import scala.collection.mutable -import akka.actor.{ LocalActorRef, Actor, ActorRef, Props, newUuid } -import akka.actor.Actor._ -import akka.actor.ActorSystemImpl +import akka.actor.{ Actor, Props, ActorSystemImpl } /** * Stream of all kinds of network events, remote failure and connection events, cluster failure and connection events etc. diff --git a/akka-remote/src/main/scala/akka/remote/Remote.scala b/akka-remote/src/main/scala/akka/remote/Remote.scala index 923ccdc85d..03479e6ee4 100644 --- a/akka-remote/src/main/scala/akka/remote/Remote.scala +++ b/akka-remote/src/main/scala/akka/remote/Remote.scala @@ -7,9 +7,7 @@ package akka.remote import akka.actor._ import akka.event._ import akka.util._ -import akka.util.duration._ -import akka.util.Helpers._ -import akka.serialization.{ JavaSerializer, Serialization, SerializationExtension } +import akka.serialization.{ Serialization, SerializationExtension } import akka.dispatch.MessageDispatcher import akka.dispatch.SystemMessage import scala.annotation.tailrec diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 378d82096b..d8cebe9c31 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -6,16 +6,9 @@ package akka.remote import akka.actor._ import akka.dispatch._ -import akka.util.duration._ -import akka.util.Timeout -import akka.config.ConfigurationException import akka.event.{ DeathWatch, Logging } -import akka.serialization.Compression.LZF -import com.google.protobuf.ByteString import akka.event.EventStream -import akka.dispatch.Promise import akka.config.ConfigurationException -import java.util.concurrent.{ TimeoutException } /** * Remote ActorRefProvider. Starts up actor on remote node and creates a RemoteActorRef representing it. diff --git a/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala b/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala index deed34521a..43c07e6968 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala @@ -6,7 +6,6 @@ package akka.remote import akka.actor._ import akka.routing._ -import akka.actor.ActorSystem import akka.event.Logging import scala.collection.immutable.Map diff --git a/akka-remote/src/main/scala/akka/remote/RemoteInterface.scala b/akka-remote/src/main/scala/akka/remote/RemoteInterface.scala index a5474ce427..3f212f51f5 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteInterface.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteInterface.scala @@ -7,13 +7,10 @@ package akka.remote import akka.actor._ import akka.AkkaException import scala.reflect.BeanProperty -import java.io.{ PrintWriter, PrintStream } -import java.net.InetSocketAddress import java.net.URI import java.net.URISyntaxException import java.net.InetAddress import java.net.UnknownHostException -import java.net.UnknownServiceException import akka.event.Logging /** diff --git a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala index 8a9a55a679..a1227ff69c 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteSettings.scala @@ -8,7 +8,6 @@ import akka.util.Duration import java.util.concurrent.TimeUnit.MILLISECONDS import java.net.InetAddress import akka.config.ConfigurationException -import com.eaio.uuid.UUID import scala.collection.JavaConverters._ class RemoteSettings(val config: Config, val systemName: String) { diff --git a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala index 4e80c84fb5..b962426577 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -4,7 +4,7 @@ package akka.remote.netty -import akka.actor.{ ActorRef, IllegalActorStateException, simpleName } +import akka.actor.{ ActorRef, simpleName } import akka.remote._ import RemoteProtocol._ import akka.util._ @@ -25,7 +25,6 @@ import akka.actor.ActorSystemImpl import org.jboss.netty.handler.execution.{ ExecutionHandler, OrderedMemoryAwareThreadPoolExecutor } import java.util.concurrent._ import locks.ReentrantReadWriteLock -import akka.dispatch.MonitorableThreadFactory class RemoteClientMessageBufferException(message: String, cause: Throwable = null) extends AkkaException(message, cause) { def this(msg: String) = this(msg, null) diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala index bbf653a808..57c949a655 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala @@ -1,11 +1,8 @@ package akka.remote -import akka.remote._ -import akka.routing._ import akka.actor.{ Actor, Props } import akka.testkit._ import akka.dispatch.Await -import akka.pattern.ask object DirectRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { override def NrOfNodes = 2 diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala index 985e8bf835..563fbc4049 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala @@ -1,12 +1,8 @@ package akka.remote import akka.actor.{ Actor, Props } -import akka.remote._ -import akka.routing._ import akka.testkit._ -import akka.util.duration._ import akka.dispatch.Await -import akka.pattern.ask object NewRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { override def NrOfNodes = 2 diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala index 2c2f0c154f..3e1339b8d4 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala @@ -1,11 +1,9 @@ package akka.remote import akka.actor.{ Actor, Props } -import akka.remote._ import akka.routing._ import akka.testkit.DefaultTimeout import akka.dispatch.Await -import akka.pattern.ask object RandomRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { override def NrOfNodes = 4 diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala index f8ae8f635c..7a7b6dce95 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala @@ -1,11 +1,9 @@ package akka.remote import akka.actor.{ Actor, Props } -import akka.remote._ import akka.routing._ import akka.testkit.DefaultTimeout import akka.dispatch.Await -import akka.pattern.ask object RoundRobinRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { override def NrOfNodes = 4 diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/ScatterGatherRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/ScatterGatherRoutedRemoteActorMultiJvmSpec.scala index 48ef5adcc9..ce9e483c07 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/ScatterGatherRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/ScatterGatherRoutedRemoteActorMultiJvmSpec.scala @@ -1,7 +1,6 @@ package akka.remote import akka.actor.{ Actor, Props } -import akka.remote._ import akka.routing._ import akka.testkit._ import akka.util.duration._ From 09a7ca2026bbe9b4566f02950220a1f2ad6be9ad Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 25 Jan 2012 15:57:19 +0100 Subject: [PATCH 143/152] Removing dead imports from sbt plugin --- akka-sbt-plugin/src/main/scala/AkkaKernelPlugin.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/akka-sbt-plugin/src/main/scala/AkkaKernelPlugin.scala b/akka-sbt-plugin/src/main/scala/AkkaKernelPlugin.scala index 5031723f44..5c1c5045af 100644 --- a/akka-sbt-plugin/src/main/scala/AkkaKernelPlugin.scala +++ b/akka-sbt-plugin/src/main/scala/AkkaKernelPlugin.scala @@ -11,7 +11,6 @@ import sbt.classpath.ClasspathUtilities import sbt.Project.Initialize import sbt.CommandSupport._ import java.io.File -import scala.collection.mutable.{ Set ⇒ MutableSet } object AkkaKernelPlugin extends Plugin { From 2e834b08e27be6ccf2c2459fa825b05af20112f3 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 25 Jan 2012 15:59:37 +0100 Subject: [PATCH 144/152] Removing dead imports from testkit --- .../main/scala/akka/testkit/CallingThreadDispatcher.scala | 1 - akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala | 4 ---- .../src/main/scala/akka/testkit/TestEventListener.scala | 2 -- akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala | 5 +---- 4 files changed, 1 insertion(+), 11 deletions(-) diff --git a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala index 96d9f8241a..af2c44611a 100644 --- a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala +++ b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala @@ -11,7 +11,6 @@ import scala.annotation.tailrec import com.typesafe.config.Config -import CallingThreadDispatcher.Id import akka.actor.{ ExtensionIdProvider, ExtensionId, Extension, ExtendedActorSystem, ActorRef, ActorCell } import akka.dispatch.{ TaskInvocation, SystemMessage, Suspend, Resume, MessageDispatcherConfigurator, MessageDispatcher, Mailbox, Envelope, DispatcherPrerequisites, DefaultSystemMessageQueue } import akka.util.duration.intToDurationInt diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala index 7b3ee7ebce..18618a8f0c 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala @@ -6,11 +6,7 @@ package akka.testkit import akka.actor._ import akka.util.{ ReflectiveAccess, Duration } -import com.eaio.uuid.UUID -import akka.actor.Props._ -import akka.actor.ActorSystem import java.util.concurrent.atomic.AtomicLong -import akka.event.EventStream import scala.collection.immutable.Stack import akka.dispatch._ import akka.pattern.ask diff --git a/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala b/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala index d191b11eab..56ced17370 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestEventListener.scala @@ -3,13 +3,11 @@ */ package akka.testkit -import scala.annotation.tailrec import scala.util.matching.Regex import akka.actor.{ DeadLetter, ActorSystem, Terminated } import akka.dispatch.{ SystemMessage, Terminate } import akka.event.Logging.{ Warning, LogEvent, InitializeLogger, Info, Error, Debug, LoggerInitialized } import akka.event.Logging -import akka.testkit.TestEvent.{ UnMute, Mute } import akka.util.Duration /** diff --git a/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala index fd3567f19c..f6d0ecfbce 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala @@ -6,10 +6,7 @@ package akka.testkit import akka.actor._ import akka.util._ -import com.eaio.uuid.UUID -import akka.actor.ActorSystem -import akka.event.EventStream -import akka.dispatch.{ DispatcherPrerequisites, Mailbox } +import akka.dispatch.DispatcherPrerequisites /** * This is a specialised form of the TestActorRef with support for querying and From f85070c68a1abaeeac02e1286495b73130d78b0b Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 25 Jan 2012 16:00:43 +0100 Subject: [PATCH 145/152] Removing dead imports for transactors --- .../src/main/scala/akka/transactor/UntypedTransactor.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/akka-transactor/src/main/scala/akka/transactor/UntypedTransactor.scala b/akka-transactor/src/main/scala/akka/transactor/UntypedTransactor.scala index ce77959a4f..0ffbd5e65c 100644 --- a/akka-transactor/src/main/scala/akka/transactor/UntypedTransactor.scala +++ b/akka-transactor/src/main/scala/akka/transactor/UntypedTransactor.scala @@ -6,7 +6,6 @@ package akka.transactor import akka.actor.{ UntypedActor, ActorRef } import scala.collection.JavaConversions._ -import scala.concurrent.stm.InTxn import java.util.{ Set ⇒ JSet } /** From 2f5e58f0922704ca01b74405a3c42bcf3971c496 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 25 Jan 2012 16:02:03 +0100 Subject: [PATCH 146/152] Removing dead imports from 0mq --- .../src/main/scala/akka/zeromq/ConcurrentSocketActor.scala | 1 - akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala | 1 - 2 files changed, 2 deletions(-) diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala index ad6cf0ff72..254a097d80 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ConcurrentSocketActor.scala @@ -8,7 +8,6 @@ import org.zeromq.{ ZMQ ⇒ JZMQ } import akka.actor._ import akka.dispatch.{ Promise, Future } import akka.event.Logging -import akka.util.duration._ import annotation.tailrec import akka.util.Duration import java.util.concurrent.TimeUnit diff --git a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala index c0486fa3ab..20dbb0724a 100644 --- a/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala +++ b/akka-zeromq/src/main/scala/akka/zeromq/ZeroMQExtension.scala @@ -3,7 +3,6 @@ */ package akka.zeromq -import akka.util.duration._ import org.zeromq.{ ZMQ ⇒ JZMQ } import akka.actor._ import akka.dispatch.{ Await } From fb03dd97dc8112c4ef701fbfe64c9fbc4d288dce Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 25 Jan 2012 16:12:28 +0100 Subject: [PATCH 147/152] Fixing multi-jvm tests --- .../scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala | 1 + .../multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala | 1 + .../scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala | 1 + .../akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala | 1 + 4 files changed, 4 insertions(+) diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala index 57c949a655..5f1c806998 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala @@ -3,6 +3,7 @@ package akka.remote import akka.actor.{ Actor, Props } import akka.testkit._ import akka.dispatch.Await +import akka.pattern.ask object DirectRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { override def NrOfNodes = 2 diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala index 563fbc4049..cef76e1b5a 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala @@ -3,6 +3,7 @@ package akka.remote import akka.actor.{ Actor, Props } import akka.testkit._ import akka.dispatch.Await +import akka.pattern.ask object NewRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { override def NrOfNodes = 2 diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala index 3e1339b8d4..887493a5fa 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala @@ -4,6 +4,7 @@ import akka.actor.{ Actor, Props } import akka.routing._ import akka.testkit.DefaultTimeout import akka.dispatch.Await +import akka.pattern.ask object RandomRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { override def NrOfNodes = 4 diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala index 7a7b6dce95..d699a41585 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala @@ -4,6 +4,7 @@ import akka.actor.{ Actor, Props } import akka.routing._ import akka.testkit.DefaultTimeout import akka.dispatch.Await +import akka.pattern.ask object RoundRobinRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { override def NrOfNodes = 4 From 9de08bb4ce191b733e5cabb9326fd1677ebed74c Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 25 Jan 2012 16:23:06 +0100 Subject: [PATCH 148/152] Review comments addressed --- akka-actor/src/main/scala/akka/dispatch/Future.scala | 2 +- akka-actor/src/main/scala/akka/util/ByteString.scala | 5 ++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 2663aff23e..495faba5d6 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -10,7 +10,7 @@ import akka.japi.{ Function ⇒ JFunc, Option ⇒ JOption } import scala.util.continuations._ -import java.util.concurrent.TimeUnit.{ NANOSECONDS } +import java.util.concurrent.TimeUnit.NANOSECONDS import java.lang.{ Iterable ⇒ JIterable } import java.util.{ LinkedList ⇒ JLinkedList } diff --git a/akka-actor/src/main/scala/akka/util/ByteString.scala b/akka-actor/src/main/scala/akka/util/ByteString.scala index dd38fe3bdd..ce23d0f354 100644 --- a/akka-actor/src/main/scala/akka/util/ByteString.scala +++ b/akka-actor/src/main/scala/akka/util/ByteString.scala @@ -1,12 +1,11 @@ package akka.util import java.nio.ByteBuffer -import java.nio.charset.Charset -import scala.collection.{ IndexedSeqOptimized } +import scala.collection.IndexedSeqOptimized import scala.collection.mutable.{ Builder, WrappedArray } import scala.collection.immutable.{ IndexedSeq, VectorBuilder } -import scala.collection.generic.{ CanBuildFrom } +import scala.collection.generic.CanBuildFrom object ByteString { From a79cbf506815b0de05da0e2234358799aafc0dcf Mon Sep 17 00:00:00 2001 From: Havoc Pennington Date: Wed, 25 Jan 2012 10:24:16 -0500 Subject: [PATCH 149/152] Copy in config lib v0.2.1 commit 2c8bf3be7 --- .../config/impl/AbstractConfigValue.java | 12 ++++++---- .../config/impl/ConfigSubstitution.java | 22 +++++++++++++++---- 2 files changed, 26 insertions(+), 8 deletions(-) diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigValue.java b/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigValue.java index 46063f410e..83d4ccdc44 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigValue.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/AbstractConfigValue.java @@ -103,6 +103,13 @@ abstract class AbstractConfigValue implements ConfigValue, MergeableValue { throw badMergeException(); } + protected AbstractConfigValue mergedWithNonObject(AbstractConfigValue fallback) { + // falling back to a non-object doesn't merge anything, and also + // prohibits merging any objects that we fall back to later. + // so we have to switch to ignoresFallbacks mode. + return newCopy(true /* ignoresFallbacks */, origin); + } + public AbstractConfigValue withOrigin(ConfigOrigin origin) { if (this.origin == origin) return this; @@ -130,10 +137,7 @@ abstract class AbstractConfigValue implements ConfigValue, MergeableValue { return mergedWithObject((AbstractConfigObject) other); } } else { - // falling back to a non-object doesn't merge anything, and also - // prohibits merging any objects that we fall back to later. - // so we have to switch to ignoresFallbacks mode. - return newCopy(true /* ignoresFallbacks */, origin); + return mergedWithNonObject((AbstractConfigValue) other); } } } diff --git a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigSubstitution.java b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigSubstitution.java index 23776e0011..e882061f11 100644 --- a/akka-actor/src/main/java/com/typesafe/config/impl/ConfigSubstitution.java +++ b/akka-actor/src/main/java/com/typesafe/config/impl/ConfigSubstitution.java @@ -84,13 +84,10 @@ final class ConfigSubstitution extends AbstractConfigValue implements ((AbstractConfigValue) fallback).ignoresFallbacks()); } - @Override - protected AbstractConfigValue mergedWithObject(AbstractConfigObject fallback) { + protected AbstractConfigValue mergedLater(AbstractConfigValue fallback) { if (ignoresFallbacks) throw new ConfigException.BugOrBroken("should not be reached"); - // if we turn out to be an object, and the fallback also does, - // then a merge may be required; delay until we resolve. List newStack = new ArrayList(); newStack.add(this); newStack.add(fallback); @@ -98,6 +95,23 @@ final class ConfigSubstitution extends AbstractConfigValue implements fallback.ignoresFallbacks()); } + @Override + protected AbstractConfigValue mergedWithObject(AbstractConfigObject fallback) { + // if we turn out to be an object, and the fallback also does, + // then a merge may be required; delay until we resolve. + return mergedLater(fallback); + } + + @Override + protected AbstractConfigValue mergedWithNonObject(AbstractConfigValue fallback) { + // if the optional substitution ends up getting deleted (because it is + // not present), we'll have to use the fallback. So delay the merge. + if (pieces.size() == 1 && ((SubstitutionExpression) pieces.get(0)).optional()) + return mergedLater(fallback); + else + return super.mergedWithNonObject(fallback); + } + @Override public Collection unmergedValues() { return Collections.singleton(this); From 91a63484978d11b82dcf0e300639044199be619d Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 25 Jan 2012 16:50:06 +0100 Subject: [PATCH 150/152] Fixing so that RemoteConfigSpec actually tests what is used by Akka --- .../scala/akka/remote/RemoteConfigSpec.scala | 74 +++++++++++-------- 1 file changed, 44 insertions(+), 30 deletions(-) diff --git a/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala b/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala index 32db14194a..57dc7f210a 100644 --- a/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemoteConfigSpec.scala @@ -1,50 +1,64 @@ package akka.remote import akka.testkit.AkkaSpec +import akka.actor.ExtendedActorSystem +import akka.util.duration._ @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) -class RemoteConfigSpec extends AkkaSpec("") { +class RemoteConfigSpec extends AkkaSpec( + """ + akka { + actor { + provider = "akka.remote.RemoteActorRefProvider" + } + } + """) { "RemoteExtension" must { "be able to parse remote and cluster config elements" in { + val settings = system.asInstanceOf[ExtendedActorSystem].provider.asInstanceOf[RemoteActorRefProvider].remoteSettings - val config = system.settings.config - import config._ + //SharedSettings - //akka.remote - getString("akka.remote.transport") must equal("akka.remote.netty.NettyRemoteSupport") - getString("akka.remote.secure-cookie") must equal("") - getBoolean("akka.remote.use-passive-connections") must equal(true) - getMilliseconds("akka.remote.backoff-timeout") must equal(0) - // getMilliseconds("akka.remote.remote-daemon-ack-timeout") must equal(30 * 1000) + { + import settings._ - //akka.remote.server - getInt("akka.remote.server.port") must equal(2552) - getBytes("akka.remote.server.message-frame-size") must equal(1048576L) + RemoteTransport must equal("akka.remote.netty.NettyRemoteSupport") + BackoffTimeout must equal(0 seconds) + LogReceivedMessages must equal(false) + LogSentMessages must equal(false) + } - getBoolean("akka.remote.server.require-cookie") must equal(false) - getBoolean("akka.remote.server.untrusted-mode") must equal(false) - getInt("akka.remote.server.backlog") must equal(4096) + //ServerSettings - getBoolean("akka.remote.log-received-messages") must equal(false) - getBoolean("akka.remote.log-sent-messages") must equal(false) + { + import settings.serverSettings._ + SecureCookie must be(None) + UsePassiveConnections must equal(true) + Port must equal(2552) + MessageFrameSize must equal(1048576L) + RequireCookie must equal(false) + UntrustedMode must equal(false) + Backlog must equal(4096) + ExecutionPoolKeepAlive must equal(1 minute) + ExecutionPoolSize must equal(4) + MaxChannelMemorySize must equal(0) + MaxTotalMemorySize must equal(0) + } - getMilliseconds("akka.remote.server.execution-pool-keepalive") must equal(60 * 1000) + //ClientSettings - getInt("akka.remote.server.execution-pool-size") must equal(4) - - getBytes("akka.remote.server.max-channel-memory-size") must equal(0) - getBytes("akka.remote.server.max-total-memory-size") must equal(0) - - //akka.remote.client - getMilliseconds("akka.remote.client.reconnect-delay") must equal(5 * 1000) - getMilliseconds("akka.remote.client.read-timeout") must equal(3600 * 1000) - getMilliseconds("akka.remote.client.reconnection-time-window") must equal(600 * 1000) - getMilliseconds("akka.remote.client.connection-timeout") must equal(10000) + { + import settings.clientSettings._ + SecureCookie must be(None) + ReconnectDelay must equal(5 seconds) + ReadTimeout must equal(1 hour) + ReconnectionTimeWindow must equal(10 minutes) + ConnectionTimeout must equal(10 seconds) + } // TODO cluster config will go into akka-cluster/reference.conf when we enable that module - //akka.cluster - getStringList("akka.cluster.seed-nodes") must equal(new java.util.ArrayList[String]) + settings.SeedNodes must be('empty) } } } From c8cd25e913f3f5515af0c5a427ee13d79e18193f Mon Sep 17 00:00:00 2001 From: Eugene Vigdorchik Date: Wed, 25 Jan 2012 20:12:26 +0400 Subject: [PATCH 151/152] Factor out localhost and re-enable schoir. --- .../akka/remote/AbstractRemoteActorMultiJvmSpec.scala | 11 ++++++----- .../remote/DirectRoutedRemoteActorMultiJvmSpec.scala | 10 ++++++---- .../akka/remote/NewRemoteActorMultiJvmSpec.scala | 4 ++-- .../remote/RandomRoutedRemoteActorMultiJvmSpec.scala | 8 ++++---- .../RoundRobinRoutedRemoteActorMultiJvmSpec.scala | 8 ++++---- .../ScatterGatherRoutedRemoteActorMultiJvmSpec.scala | 4 ++-- project/AkkaBuild.scala | 4 ++-- project/plugins.sbt | 2 +- 8 files changed, 27 insertions(+), 24 deletions(-) diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/AbstractRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/AbstractRemoteActorMultiJvmSpec.scala index 597b552fe9..4067706ba9 100755 --- a/akka-remote/src/multi-jvm/scala/akka/remote/AbstractRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/AbstractRemoteActorMultiJvmSpec.scala @@ -6,14 +6,15 @@ trait AbstractRemoteActorMultiJvmSpec { def NrOfNodes: Int def commonConfig: Config - def remotes: Seq[String] = { + private[this] val remotes: Array[String] = { val arrayOpt = Option(AkkaRemoteSpec.testNodes).map(_ split ",") - (arrayOpt getOrElse Array.fill(NrOfNodes)("localhost")).toSeq + (arrayOpt getOrElse Array.fill(NrOfNodes)("localhost")).toArray } - def specString(count: Int): String = { - val specs = for ((host, idx) <- remotes.take(count).zipWithIndex) yield - "\"akka://AkkaRemoteSpec@%s:%d\"".format(host, 9991+idx) + def akkaSpec(idx: Int) = "AkkaRemoteSpec@%s:%d".format(remotes(idx), 9991+idx) + + def akkaURIs(count: Int): String = { + val specs = for (idx <- 0 until count) yield "\"akka://" + akkaSpec(idx) + "\"" specs.mkString(",") } diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala index bbf653a808..0ac13c4f12 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/DirectRoutedRemoteActorMultiJvmSpec.scala @@ -26,10 +26,12 @@ object DirectRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSp /service-hello.remote = %s } } - }""" format specString(1)) + }""" format akkaURIs(1)) } -class DirectRoutedRemoteActorMultiJvmNode1 extends AkkaRemoteSpec(DirectRoutedRemoteActorMultiJvmSpec.nodeConfigs(0)) { +import DirectRoutedRemoteActorMultiJvmSpec._ + +class DirectRoutedRemoteActorMultiJvmNode1 extends AkkaRemoteSpec(nodeConfigs(0)) { import DirectRoutedRemoteActorMultiJvmSpec._ val nodes = NrOfNodes @@ -41,7 +43,7 @@ class DirectRoutedRemoteActorMultiJvmNode1 extends AkkaRemoteSpec(DirectRoutedRe } } -class DirectRoutedRemoteActorMultiJvmNode2 extends AkkaRemoteSpec(DirectRoutedRemoteActorMultiJvmSpec.nodeConfigs(1)) with DefaultTimeout { +class DirectRoutedRemoteActorMultiJvmNode2 extends AkkaRemoteSpec(nodeConfigs(1)) with DefaultTimeout { import DirectRoutedRemoteActorMultiJvmSpec._ val nodes = NrOfNodes @@ -53,7 +55,7 @@ class DirectRoutedRemoteActorMultiJvmNode2 extends AkkaRemoteSpec(DirectRoutedRe val actor = system.actorOf(Props[SomeActor], "service-hello") actor.isInstanceOf[RemoteActorRef] must be(true) - Await.result(actor ? "identify", timeout.duration) must equal("AkkaRemoteSpec@localhost:9991") + Await.result(actor ? "identify", timeout.duration) must equal(akkaSpec(0)) barrier("done") } diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala index 985e8bf835..03ee68e616 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/NewRemoteActorMultiJvmSpec.scala @@ -27,7 +27,7 @@ object NewRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSpec { /service-hello.remote = %s } } - }""" format specString(1)) + }""" format akkaURIs(1)) } class NewRemoteActorMultiJvmNode1 extends AkkaRemoteSpec(NewRemoteActorMultiJvmSpec.nodeConfigs(0)) { @@ -56,7 +56,7 @@ class NewRemoteActorMultiJvmNode2 extends AkkaRemoteSpec(NewRemoteActorMultiJvmS barrier("start") val actor = system.actorOf(Props[SomeActor], "service-hello") - Await.result(actor ? "identify", timeout.duration) must equal("AkkaRemoteSpec@localhost:9991") + Await.result(actor ? "identify", timeout.duration) must equal(akkaSpec(0)) barrier("done") } diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala index 2c2f0c154f..2fc6b413e8 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/RandomRoutedRemoteActorMultiJvmSpec.scala @@ -28,7 +28,7 @@ object RandomRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJvmSp /service-hello.target.nodes = [%s] } } - }""" format (3, specString(3))) + }""" format (3, akkaURIs(3))) } class RandomRoutedRemoteActorMultiJvmNode1 extends AkkaRemoteSpec(RandomRoutedRemoteActorMultiJvmSpec.nodeConfigs(0)) { @@ -84,9 +84,9 @@ class RandomRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec(RandomRoutedRe val iterationCount = 10 var replies = Map( - "AkkaRemoteSpec@localhost:9991" -> 0, - "AkkaRemoteSpec@localhost:9992" -> 0, - "AkkaRemoteSpec@localhost:9993" -> 0) + akkaSpec(0) -> 0, + akkaSpec(1) -> 0, + akkaSpec(2) -> 0) for (i ← 0 until iterationCount) { for (k ← 0 until connectionCount) { diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala index f8ae8f635c..4f4dae7f70 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/RoundRobinRoutedRemoteActorMultiJvmSpec.scala @@ -29,7 +29,7 @@ object RoundRobinRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMultiJ /service-hello.target.nodes = [%s] } } - }""" format (3, specString(3))) + }""" format (3, akkaURIs(3))) } class RoundRobinRoutedRemoteActorMultiJvmNode1 extends AkkaRemoteSpec(RoundRobinRoutedRemoteActorMultiJvmSpec.nodeConfigs(0)) { @@ -85,9 +85,9 @@ class RoundRobinRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec(RoundRobin val iterationCount = 10 var replies = Map( - "AkkaRemoteSpec@localhost:9991" -> 0, - "AkkaRemoteSpec@localhost:9992" -> 0, - "AkkaRemoteSpec@localhost:9993" -> 0) + akkaSpec(0) -> 0, + akkaSpec(1) -> 0, + akkaSpec(2) -> 0) for (i ← 0 until iterationCount) { for (k ← 0 until connectionCount) { diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/ScatterGatherRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/ScatterGatherRoutedRemoteActorMultiJvmSpec.scala index 48ef5adcc9..678357a57c 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/ScatterGatherRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/ScatterGatherRoutedRemoteActorMultiJvmSpec.scala @@ -27,7 +27,7 @@ object ScatterGatherRoutedRemoteActorMultiJvmSpec extends AbstractRemoteActorMul /service-hello.target.nodes = [%s] } } - }""" format (3, specString(3))) + }""" format (3, akkaURIs(3))) } class ScatterGatherRoutedRemoteActorMultiJvmNode1 extends AkkaRemoteSpec(ScatterGatherRoutedRemoteActorMultiJvmSpec.nodeConfigs(0)) { @@ -91,7 +91,7 @@ class ScatterGatherRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec(Scatter val replies = (receiveWhile(5 seconds, messages = connectionCount * iterationCount) { case name: String ⇒ (name, 1) - }).foldLeft(Map("AkkaRemoteSpec@localhost:9991" -> 0, "AkkaRemoteSpec@localhost:9992" -> 0, "AkkaRemoteSpec@localhost:9993" -> 0)) { + }).foldLeft(Map(akkaSpec(0) -> 0, akkaSpec(1) -> 0, akkaSpec(2) -> 0)) { case (m, (n, c)) ⇒ m + (n -> (m(n) + c)) } diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index d104a9db5d..9dada98416 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -8,7 +8,7 @@ import sbt._ import sbt.Keys._ import com.typesafe.sbtmultijvm.MultiJvmPlugin import com.typesafe.sbtmultijvm.MultiJvmPlugin.{ MultiJvm, extraOptions, jvmOptions, scalatestOptions } -//import com.typesafe.schoir.SchoirPlugin.schoirSettings +import com.typesafe.schoir.SchoirPlugin.schoirSettings import com.typesafe.sbtscalariform.ScalariformPlugin import com.typesafe.sbtscalariform.ScalariformPlugin.ScalariformKeys import java.lang.Boolean.getBoolean @@ -71,7 +71,7 @@ object AkkaBuild extends Build { id = "akka-remote", base = file("akka-remote"), dependencies = Seq(actor, actorTests % "test->test", testkit % "test->test"), - settings = defaultSettings ++ multiJvmSettings ++ /*schoirSettings ++*/ Seq( + settings = defaultSettings ++ multiJvmSettings ++ schoirSettings ++ Seq( libraryDependencies ++= Dependencies.remote, // disable parallel tests parallelExecution in Test := false, diff --git a/project/plugins.sbt b/project/plugins.sbt index 899db6307f..52704b5935 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -3,7 +3,7 @@ resolvers += Classpaths.typesafeResolver addSbtPlugin("com.typesafe.sbtmultijvm" % "sbt-multi-jvm" % "0.1.9") -//addSbtPlugin("com.typesafe.schoir" % "schoir" % "0.1.1") +addSbtPlugin("com.typesafe.schoir" % "schoir" % "0.1.2") addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse" % "1.5.0") From 0f92df274dad41d1e22ed1ac180112ee48bcf232 Mon Sep 17 00:00:00 2001 From: Eugene Vigdorchik Date: Wed, 25 Jan 2012 20:43:03 +0400 Subject: [PATCH 152/152] Use IndexedSeq instead of Arrays. --- .../akka/remote/AbstractRemoteActorMultiJvmSpec.scala | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/AbstractRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/AbstractRemoteActorMultiJvmSpec.scala index 4067706ba9..fc6dc81175 100755 --- a/akka-remote/src/multi-jvm/scala/akka/remote/AbstractRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/AbstractRemoteActorMultiJvmSpec.scala @@ -6,16 +6,15 @@ trait AbstractRemoteActorMultiJvmSpec { def NrOfNodes: Int def commonConfig: Config - private[this] val remotes: Array[String] = { - val arrayOpt = Option(AkkaRemoteSpec.testNodes).map(_ split ",") - (arrayOpt getOrElse Array.fill(NrOfNodes)("localhost")).toArray + private[this] val remotes: IndexedSeq[String] = { + val nodesOpt = Option(AkkaRemoteSpec.testNodes).map(_.split(",").toIndexedSeq) + nodesOpt getOrElse IndexedSeq.fill(NrOfNodes)("localhost") } def akkaSpec(idx: Int) = "AkkaRemoteSpec@%s:%d".format(remotes(idx), 9991+idx) def akkaURIs(count: Int): String = { - val specs = for (idx <- 0 until count) yield "\"akka://" + akkaSpec(idx) + "\"" - specs.mkString(",") + 0 until count map {idx => "\"akka://" + akkaSpec(idx) + "\""} mkString "," } val nodeConfigs = ((1 to NrOfNodes).toList zip remotes) map {