implement ResumeWriting, see #3200
also included: - a complete rewrite of the TCP docs based on real/tested/working code samples - an EchoServer implementation which handles all the edge cases, available in Java & Scala - renamed StopReading to SuspendReading to match up with ResumeReading - addition of Inbox.watch() - Inbox RST docs for Java(!) and Scala not included: - ScalaDoc / JavaDoc for all IO stuff
This commit is contained in:
parent
489c00b913
commit
0e34edbcb3
20 changed files with 1874 additions and 187 deletions
|
|
@ -255,6 +255,24 @@ If you want to use this magic, simply extend :class:`ActWithStash`:
|
|||
|
||||
.. includecode:: ../../../akka-actor-tests/src/test/scala/akka/actor/ActorDSLSpec.scala#act-with-stash
|
||||
|
||||
The Inbox
|
||||
---------
|
||||
|
||||
When writing code outside of actors which shall communicate with actors, the
|
||||
``ask`` pattern can be a solution (see below), but there are two thing it
|
||||
cannot do: receiving multiple replies (e.g. by subscribing an :class:`ActorRef`
|
||||
to a notification service) and watching other actors’ lifecycle. For these
|
||||
purposes there is the :class:`Inbox` class:
|
||||
|
||||
.. includecode:: ../../../akka-actor-tests/src/test/scala/akka/actor/ActorDSLSpec.scala#inbox
|
||||
|
||||
There is an implicit conversion from inbox to actor reference which means that
|
||||
in this example the sender reference will be that of the actor hidden away
|
||||
within the inbox. This allows the reply to be received on the last line.
|
||||
Watching an actor is quite simple as well:
|
||||
|
||||
.. includecode:: ../../../akka-actor-tests/src/test/scala/akka/actor/ActorDSLSpec.scala#watch
|
||||
|
||||
Actor API
|
||||
=========
|
||||
|
||||
|
|
|
|||
302
akka-docs/rst/scala/code/docs/io/EchoServer.scala
Normal file
302
akka-docs/rst/scala/code/docs/io/EchoServer.scala
Normal file
|
|
@ -0,0 +1,302 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package docs.io
|
||||
|
||||
import java.net.InetSocketAddress
|
||||
|
||||
import scala.concurrent.duration.DurationInt
|
||||
|
||||
import com.typesafe.config.ConfigFactory
|
||||
|
||||
import akka.actor.{ Actor, ActorDSL, ActorLogging, ActorRef, ActorSystem, Props, SupervisorStrategy }
|
||||
import akka.actor.ActorDSL.inbox
|
||||
import akka.io.{ IO, Tcp }
|
||||
import akka.util.ByteString
|
||||
|
||||
object EchoServer extends App {
|
||||
|
||||
val config = ConfigFactory.parseString("akka.loglevel = DEBUG")
|
||||
implicit val system = ActorSystem("EchoServer", config)
|
||||
|
||||
// make sure to stop the system so that the application stops
|
||||
try run()
|
||||
finally system.shutdown()
|
||||
|
||||
def run(): Unit = {
|
||||
import ActorDSL._
|
||||
|
||||
// create two EchoManager and stop the application once one dies
|
||||
val watcher = inbox()
|
||||
watcher.watch(system.actorOf(Props(classOf[EchoManager], classOf[EchoHandler]), "echo"))
|
||||
watcher.watch(system.actorOf(Props(classOf[EchoManager], classOf[SimpleEchoHandler]), "simple"))
|
||||
watcher.receive(10.minutes)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
class EchoManager(handlerClass: Class[_]) extends Actor with ActorLogging {
|
||||
|
||||
import Tcp._
|
||||
import context.system
|
||||
|
||||
// there is not recovery for broken connections
|
||||
override val supervisorStrategy = SupervisorStrategy.stoppingStrategy
|
||||
|
||||
// bind to the listen port; the port will automatically be closed once this actor dies
|
||||
override def preStart(): Unit = {
|
||||
IO(Tcp) ! Bind(self, new InetSocketAddress("localhost", 0))
|
||||
}
|
||||
|
||||
// do not restart
|
||||
override def postRestart(thr: Throwable): Unit = context stop self
|
||||
|
||||
def receive = {
|
||||
case Bound(localAddress) ⇒
|
||||
log.info("listening on port {}", localAddress.getPort)
|
||||
|
||||
case CommandFailed(Bind(_, local, _, _)) ⇒
|
||||
log.warning(s"cannot bind to [$local]")
|
||||
context stop self
|
||||
|
||||
//#echo-manager
|
||||
case Connected(remote, local) ⇒
|
||||
log.info("received connection from {}", remote)
|
||||
val handler = context.actorOf(Props(handlerClass, sender, remote))
|
||||
sender ! Register(handler, keepOpenOnPeerClosed = true)
|
||||
//#echo-manager
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
object EchoHandler {
|
||||
def apply(connection: ActorRef, remote: InetSocketAddress): Props =
|
||||
Props(classOf[EchoHandler], connection, remote)
|
||||
}
|
||||
|
||||
//#echo-handler
|
||||
class EchoHandler(connection: ActorRef, remote: InetSocketAddress)
|
||||
extends Actor with ActorLogging {
|
||||
|
||||
import Tcp._
|
||||
|
||||
// sign death pact: this actor terminates when connection breaks
|
||||
context watch connection
|
||||
|
||||
// start out in optimistic write-through mode
|
||||
def receive = writing
|
||||
|
||||
//#writing
|
||||
def writing: Receive = {
|
||||
case Received(data) ⇒
|
||||
connection ! Write(data, currentOffset)
|
||||
buffer(data)
|
||||
|
||||
case ack: Int ⇒
|
||||
acknowledge(ack)
|
||||
|
||||
case CommandFailed(Write(_, ack: Int)) ⇒
|
||||
connection ! ResumeWriting
|
||||
context become buffering(ack)
|
||||
|
||||
case PeerClosed ⇒
|
||||
if (storage.isEmpty) context stop self
|
||||
else context become closing
|
||||
}
|
||||
//#writing
|
||||
|
||||
//#buffering
|
||||
def buffering(nack: Int): Receive = {
|
||||
var toAck = 10
|
||||
var peerClosed = false
|
||||
|
||||
{
|
||||
case Received(data) ⇒ buffer(data)
|
||||
case WritingResumed ⇒ writeFirst()
|
||||
case PeerClosed ⇒ peerClosed = true
|
||||
case ack: Int if ack < nack ⇒ acknowledge(ack)
|
||||
case ack: Int ⇒
|
||||
acknowledge(ack)
|
||||
if (storage.nonEmpty) {
|
||||
if (toAck > 0) {
|
||||
// stay in ACK-based mode for a while
|
||||
writeFirst()
|
||||
toAck -= 1
|
||||
} else {
|
||||
// then return to NACK-based again
|
||||
writeAll()
|
||||
context become (if (peerClosed) closing else writing)
|
||||
}
|
||||
} else if (peerClosed) context stop self
|
||||
else context become writing
|
||||
}
|
||||
}
|
||||
//#buffering
|
||||
|
||||
//#closing
|
||||
def closing: Receive = {
|
||||
case CommandFailed(_: Write) ⇒
|
||||
connection ! ResumeWriting
|
||||
context.become({
|
||||
|
||||
case WritingResumed ⇒
|
||||
writeAll()
|
||||
context.unbecome()
|
||||
|
||||
case ack: Int ⇒ acknowledge(ack)
|
||||
|
||||
}, discardOld = false)
|
||||
|
||||
case ack: Int ⇒
|
||||
acknowledge(ack)
|
||||
if (storage.isEmpty) context stop self
|
||||
}
|
||||
//#closing
|
||||
|
||||
override def postStop(): Unit = {
|
||||
log.info(s"transferred $transferred bytes from/to [$remote]")
|
||||
}
|
||||
|
||||
//#storage-omitted
|
||||
var storageOffset = 0
|
||||
var storage = Vector.empty[ByteString]
|
||||
var stored = 0L
|
||||
var transferred = 0L
|
||||
|
||||
val maxStored = 100000000L
|
||||
val highWatermark = maxStored * 5 / 10
|
||||
val lowWatermark = maxStored * 3 / 10
|
||||
var suspended = false
|
||||
|
||||
private def currentOffset = storageOffset + storage.size
|
||||
|
||||
//#helpers
|
||||
private def buffer(data: ByteString): Unit = {
|
||||
storage :+= data
|
||||
stored += data.size
|
||||
|
||||
if (stored > maxStored) {
|
||||
log.warning(s"drop connection to [$remote] (buffer overrun)")
|
||||
context stop self
|
||||
|
||||
} else if (stored > highWatermark) {
|
||||
log.debug(s"suspending reading at $currentOffset")
|
||||
connection ! SuspendReading
|
||||
suspended = true
|
||||
}
|
||||
}
|
||||
|
||||
private def acknowledge(ack: Int): Unit = {
|
||||
require(ack == storageOffset, s"received ack $ack at $storageOffset")
|
||||
require(storage.nonEmpty, s"storage was empty at ack $ack")
|
||||
|
||||
val size = storage(0).size
|
||||
stored -= size
|
||||
transferred += size
|
||||
|
||||
storageOffset += 1
|
||||
storage = storage drop 1
|
||||
|
||||
if (suspended && stored < lowWatermark) {
|
||||
log.debug("resuming reading")
|
||||
connection ! ResumeReading
|
||||
suspended = false
|
||||
}
|
||||
}
|
||||
//#helpers
|
||||
|
||||
private def writeFirst(): Unit = {
|
||||
connection ! Write(storage(0), storageOffset)
|
||||
}
|
||||
|
||||
private def writeAll(): Unit = {
|
||||
for ((data, i) ← storage.zipWithIndex) {
|
||||
connection ! Write(data, storageOffset + i)
|
||||
}
|
||||
}
|
||||
|
||||
//#storage-omitted
|
||||
}
|
||||
//#echo-handler
|
||||
|
||||
//#simple-echo-handler
|
||||
class SimpleEchoHandler(connection: ActorRef, remote: InetSocketAddress)
|
||||
extends Actor with ActorLogging {
|
||||
|
||||
import Tcp._
|
||||
|
||||
// sign death pact: this actor terminates when connection breaks
|
||||
context watch connection
|
||||
|
||||
case object Ack
|
||||
|
||||
def receive = {
|
||||
case Received(data) ⇒
|
||||
buffer(data)
|
||||
connection ! Write(data, Ack)
|
||||
|
||||
context.become({
|
||||
case Received(data) ⇒ buffer(data)
|
||||
case Ack ⇒ acknowledge()
|
||||
case PeerClosed ⇒ closing = true
|
||||
}, discardOld = false)
|
||||
|
||||
case PeerClosed ⇒ context stop self
|
||||
}
|
||||
|
||||
//#storage-omitted
|
||||
override def postStop(): Unit = {
|
||||
log.info(s"transferred $transferred bytes from/to [$remote]")
|
||||
}
|
||||
|
||||
var storage = Vector.empty[ByteString]
|
||||
var stored = 0L
|
||||
var transferred = 0L
|
||||
var closing = false
|
||||
|
||||
val maxStored = 100000000L
|
||||
val highWatermark = maxStored * 5 / 10
|
||||
val lowWatermark = maxStored * 3 / 10
|
||||
var suspended = false
|
||||
|
||||
//#simple-helpers
|
||||
private def buffer(data: ByteString): Unit = {
|
||||
storage :+= data
|
||||
stored += data.size
|
||||
|
||||
if (stored > maxStored) {
|
||||
log.warning(s"drop connection to [$remote] (buffer overrun)")
|
||||
context stop self
|
||||
|
||||
} else if (stored > highWatermark) {
|
||||
log.debug(s"suspending reading")
|
||||
connection ! SuspendReading
|
||||
suspended = true
|
||||
}
|
||||
}
|
||||
|
||||
private def acknowledge(): Unit = {
|
||||
require(storage.nonEmpty, "storage was empty")
|
||||
|
||||
val size = storage(0).size
|
||||
stored -= size
|
||||
transferred += size
|
||||
|
||||
storage = storage drop 1
|
||||
|
||||
if (suspended && stored < lowWatermark) {
|
||||
log.debug("resuming reading")
|
||||
connection ! ResumeReading
|
||||
suspended = false
|
||||
}
|
||||
|
||||
if (storage.isEmpty) {
|
||||
if (closing) context stop self
|
||||
else context.unbecome()
|
||||
} else connection ! Write(storage(0), Ack)
|
||||
}
|
||||
//#simple-helpers
|
||||
//#storage-omitted
|
||||
}
|
||||
//#simple-echo-handler
|
||||
117
akka-docs/rst/scala/code/docs/io/IODocSpec.scala
Normal file
117
akka-docs/rst/scala/code/docs/io/IODocSpec.scala
Normal file
|
|
@ -0,0 +1,117 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package docs.io
|
||||
|
||||
//#imports
|
||||
import akka.actor.{ Actor, ActorRef, Props }
|
||||
import akka.io.{ IO, Tcp }
|
||||
import akka.util.ByteString
|
||||
import java.net.InetSocketAddress
|
||||
//#imports
|
||||
|
||||
import akka.testkit.AkkaSpec
|
||||
import scala.concurrent.duration._
|
||||
|
||||
class DemoActor extends Actor {
|
||||
//#manager
|
||||
import akka.io.{ IO, Tcp }
|
||||
import context.system // implicitly used by IO(Tcp)
|
||||
|
||||
val manager = IO(Tcp)
|
||||
//#manager
|
||||
|
||||
def receive = Actor.emptyBehavior
|
||||
}
|
||||
|
||||
//#server
|
||||
object Server {
|
||||
def apply(manager: ActorRef) = Props(classOf[Server], manager)
|
||||
}
|
||||
|
||||
class Server(manager: ActorRef) extends Actor {
|
||||
|
||||
import Tcp._
|
||||
import context.system
|
||||
|
||||
IO(Tcp) ! Bind(self, new InetSocketAddress("localhost", 0))
|
||||
|
||||
def receive = {
|
||||
case b @ Bound(localAddress) ⇒ manager ! b
|
||||
|
||||
case CommandFailed(_: Bind) ⇒ context stop self
|
||||
|
||||
case c @ Connected(remote, local) ⇒
|
||||
manager ! c
|
||||
val handler = context.actorOf(Props[SimplisticHandler])
|
||||
val connection = sender
|
||||
connection ! Register(handler)
|
||||
}
|
||||
|
||||
}
|
||||
//#server
|
||||
|
||||
//#simplistic-handler
|
||||
class SimplisticHandler extends Actor {
|
||||
import Tcp._
|
||||
def receive = {
|
||||
case Received(data) ⇒ sender ! Write(data)
|
||||
case PeerClosed ⇒ context stop self
|
||||
}
|
||||
}
|
||||
//#simplistic-handler
|
||||
|
||||
//#client
|
||||
object Client {
|
||||
def apply(remote: InetSocketAddress, replies: ActorRef) =
|
||||
Props(classOf[Client], remote, replies)
|
||||
}
|
||||
|
||||
class Client(remote: InetSocketAddress, listener: ActorRef) extends Actor {
|
||||
|
||||
import Tcp._
|
||||
import context.system
|
||||
|
||||
IO(Tcp) ! Connect(remote)
|
||||
|
||||
def receive = {
|
||||
case CommandFailed(_: Connect) ⇒
|
||||
listener ! "failed"
|
||||
context stop self
|
||||
|
||||
case c @ Connected(remote, local) ⇒
|
||||
listener ! c
|
||||
val connection = sender
|
||||
connection ! Register(self)
|
||||
context become {
|
||||
case data: ByteString ⇒ connection ! Write(data)
|
||||
case CommandFailed(w: Write) ⇒ // O/S buffer was full
|
||||
case Received(data) ⇒ listener ! data
|
||||
case "close" ⇒ connection ! Close
|
||||
case _: ConnectionClosed ⇒ context stop self
|
||||
}
|
||||
}
|
||||
}
|
||||
//#client
|
||||
|
||||
class IODocSpec extends AkkaSpec {
|
||||
|
||||
"demonstrate connect" in {
|
||||
val server = system.actorOf(Server(testActor), "server1")
|
||||
val listen = expectMsgType[Tcp.Bound].localAddress
|
||||
val client = system.actorOf(Client(listen, testActor), "client1")
|
||||
|
||||
val c1, c2 = expectMsgType[Tcp.Connected]
|
||||
c1.localAddress must be(c2.remoteAddress)
|
||||
c2.localAddress must be(c1.remoteAddress)
|
||||
|
||||
client ! ByteString("hello")
|
||||
expectMsgType[ByteString].utf8String must be("hello")
|
||||
|
||||
watch(client)
|
||||
client ! "close"
|
||||
expectTerminated(client, 1.second)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -11,22 +11,26 @@ and `spray.io`_ teams. Its design combines experiences from the
|
|||
``spray-io`` module with improvements that were jointly developed for
|
||||
more general consumption as an actor-based service.
|
||||
|
||||
This documentation is in progress and some sections may be incomplete. More will be coming.
|
||||
The guiding design goal for this I/O implementation was to reach extreme
|
||||
scalability, make no compromises in providing an API correctly matching the
|
||||
underlying transport mechanism and to be fully event-driven, non-blocking and
|
||||
asynchronous. The API is meant to be a solid foundation for the implementation
|
||||
of network protocols and building higher abstractions; it is not meant to be a
|
||||
full-service high-level NIO wrapper for end users.
|
||||
|
||||
.. note::
|
||||
The old I/O implementation has been deprecated and its documentation has been moved: :ref:`io-scala-old`
|
||||
|
||||
Terminology, Concepts
|
||||
---------------------
|
||||
|
||||
The I/O API is completely actor based, meaning that all operations are implemented with message passing instead of
|
||||
direct method calls. Every I/O driver (TCP, UDP) has a special actor, called a *manager* that serves
|
||||
as an entry point for the API. I/O is broken into several drivers. The manager for a particular driver
|
||||
is accessible through the ``IO`` entry point. For example the following code
|
||||
looks up the TCP manager and returns its ``ActorRef``:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
val tcpManager = IO(Tcp)
|
||||
.. includecode:: code/docs/io/IODocSpec.scala#manager
|
||||
|
||||
The manager receives I/O command messages and instantiates worker actors in response. The worker actors present
|
||||
themselves to the API user in the reply to the command that was sent. For example after a ``Connect`` command sent to
|
||||
|
|
@ -366,107 +370,92 @@ this must be modeled either as a command or event, i.e. it will be part of the
|
|||
Using TCP
|
||||
---------
|
||||
|
||||
The code snippets through-out this section assume the following imports:
|
||||
|
||||
.. includecode:: code/docs/io/IODocSpec.scala#imports
|
||||
|
||||
All of the Akka I/O APIs are accessed through manager objects. When using an I/O API, the first step is to acquire a
|
||||
reference to the appropriate manager. The code below shows how to acquire a reference to the ``Tcp`` manager.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.io.IO
|
||||
import akka.io.Tcp
|
||||
val tcpManager = IO(Tcp)
|
||||
.. includecode:: code/docs/io/IODocSpec.scala#manager
|
||||
|
||||
The manager is an actor that handles the underlying low level I/O resources (selectors, channels) and instantiates
|
||||
workers for specific tasks, such as listening to incoming connections.
|
||||
|
||||
.. _connecting-scala:
|
||||
|
||||
Connecting
|
||||
^^^^^^^^^^
|
||||
|
||||
The first step of connecting to a remote address is sending a ``Connect`` message to the TCP manager:
|
||||
.. includecode:: code/docs/io/IODocSpec.scala#client
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
import akka.io.Tcp._
|
||||
IO(Tcp) ! Connect(remoteSocketAddress)
|
||||
|
||||
When connecting, it is also possible to set various socket options or specify a local address:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
IO(Tcp) ! Connect(remoteSocketAddress, Some(localSocketAddress), List(SO.KeepAlive(true)))
|
||||
The first step of connecting to a remote address is sending a :class:`Connect`
|
||||
message to the TCP manager; in addition to the simplest form shown above there
|
||||
is also the possibility to specify a local :class:`InetSocketAddress` to bind
|
||||
to and a list of socket options to apply.
|
||||
|
||||
.. note::
|
||||
The SO_NODELAY (TCP_NODELAY on Windows) socket option defaults to true in Akka, independently of the OS default
|
||||
settings. This setting disables Nagle's algorithm considerably improving latency for most applications. This setting
|
||||
could be overridden by passing ``SO.TcpNoDelay(false)`` in the list of socket options of the ``Connect`` message.
|
||||
|
||||
After issuing the ``Connect`` command the TCP manager spawns a worker actor to handle commands related to the
|
||||
connection. This worker actor will reveal itself by replying with a ``Connected`` message to the actor who sent the
|
||||
``Connect`` command.
|
||||
The SO_NODELAY (TCP_NODELAY on Windows) socket option defaults to true in
|
||||
Akka, independently of the OS default settings. This setting disables Nagle's
|
||||
algorithm, considerably improving latency for most applications. This setting
|
||||
could be overridden by passing ``SO.TcpNoDelay(false)`` in the list of socket
|
||||
options of the ``Connect`` message.
|
||||
|
||||
.. code-block:: scala
|
||||
The TCP manager will then reply either with a :class:`CommandFailed` or it will
|
||||
spawn an internal actor representing the new connection. This new actor will
|
||||
then send a :class:`Connected` message to the original sender of the
|
||||
:class:`Connect` message.
|
||||
|
||||
case Connected(remoteAddress, localAddress) =>
|
||||
connectionActor = sender
|
||||
In order to activate the new connection a :class:`Register` message must be
|
||||
sent to the connection actor, informing that one about who shall receive data
|
||||
from the socket. Before this step is done the connection cannot be used, and
|
||||
there is an internal timeout after which the connection actor will shut itself
|
||||
down if no :class:`Register` message is received.
|
||||
|
||||
At this point, there is still no listener associated with the connection. To finish the connection setup a ``Register``
|
||||
has to be sent to the connection actor with the listener ``ActorRef`` as a parameter.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
connectionActor ! Register(listener)
|
||||
|
||||
Upon registration, the connection actor will watch the listener actor provided in the ``listener`` parameter.
|
||||
If the listener actor stops, the connection is closed, and all resources allocated for the connection released. During the
|
||||
lifetime of the connection the listener may receive various event notifications:
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
case Received(dataByteString) => // handle incoming chunk of data
|
||||
case CommandFailed(cmd) => // handle failure of command: cmd
|
||||
case _: ConnectionClosed => // handle closed connections
|
||||
|
||||
``ConnectionClosed`` is a trait, which the different connection close events all implement.
|
||||
The last line handles all connection close events in the same way. It is possible to listen for more fine-grained
|
||||
connection close events, see :ref:`closing-connections-scala` below.
|
||||
The connection actor watches the registered handler and closes the connection
|
||||
when that one terminates, thereby cleaning up all internal resources associated
|
||||
with that connection.
|
||||
|
||||
The actor in the example above uses :meth:`become` to switch from unconnected
|
||||
to connected operation, demonstrating the commands and events which are
|
||||
observed in that state. For a discussion on :class:`CommandFailed` see
|
||||
`Throttling Reads and Writes`_ below. :class:`ConnectionClosed` is a trait,
|
||||
which marks the different connection close events. The last line handles all
|
||||
connection close events in the same way. It is possible to listen for more
|
||||
fine-grained connection close events, see `Closing Connections`_ below.
|
||||
|
||||
Accepting connections
|
||||
^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
To create a TCP server and listen for inbound connections, a ``Bind`` command has to be sent to the TCP manager.
|
||||
This will instruct the TCP manager to listen for TCP connections on a particular address.
|
||||
.. includecode:: code/docs/io/IODocSpec.scala#server
|
||||
|
||||
.. code-block:: scala
|
||||
To create a TCP server and listen for inbound connections, a :class:`Bind`
|
||||
command has to be sent to the TCP manager. This will instruct the TCP manager
|
||||
to listen for TCP connections on a particular :class:`InetSocketAddress`; the
|
||||
port may be specified as ``0`` in order to bind to a random port.
|
||||
|
||||
import akka.io.IO
|
||||
import akka.io.Tcp
|
||||
IO(Tcp) ! Bind(handler, localAddress)
|
||||
The actor sending the :class:`Bind` message will receive a :class:`Bound`
|
||||
message signalling that the server is ready to accept incoming connections;
|
||||
this message also contains the :class:`InetSocketAddress` to which the socket
|
||||
was actually bound (i.e. resolved IP address and correct port number).
|
||||
|
||||
The actor sending the ``Bind`` message will receive a ``Bound`` message signalling that the server is ready to accept
|
||||
incoming connections. The process for accepting connections is similar to the process for making :ref:`outgoing
|
||||
connections <connecting-scala>`: when an incoming connection is established, the actor provided as ``handler`` will
|
||||
receive a ``Connected`` message whose sender is the connection actor.
|
||||
From this point forward the process of handling connections is the same as for
|
||||
outgoing connections. The example demonstrates that handling the reads from a
|
||||
certain connection can be delegated to another actor by naming it as the
|
||||
handler when sending the :class:`Register` message. Writes can be sent from any
|
||||
actor in the system to the connection actor (i.e. the actor which sent the
|
||||
:class:`Connected` message). The simplistic handler is defined as:
|
||||
|
||||
.. code-block:: scala
|
||||
.. includecode:: code/docs/io/IODocSpec.scala#simplistic-handler
|
||||
|
||||
case Connected(remoteAddress, localAddress) =>
|
||||
connectionActor = sender
|
||||
For a more complete sample which also takes into account the possibility of
|
||||
failures when sending please see `Throttling Reads and Writes`_ below.
|
||||
|
||||
At this point, there is still no listener associated with the connection. To finish the connection setup a ``Register``
|
||||
has to be sent to the connection actor with the listener ``ActorRef`` as a parameter.
|
||||
|
||||
.. code-block:: scala
|
||||
|
||||
connectionActor ! Register(listener)
|
||||
|
||||
Upon registration, the connection actor will watch the listener actor provided in the ``listener`` parameter.
|
||||
If the listener stops, the connection is closed, and all resources allocated for the connection are released. During the
|
||||
connection lifetime the listener will receive various event notifications in the same way as in the outbound
|
||||
connection case.
|
||||
|
||||
.. _closing-connections-scala:
|
||||
The only difference to outgoing connections is that the internal actor managing
|
||||
the listen port—the sender of the :class:`Bound` message—watches the actor
|
||||
which was named as the recipient for :class:`Connected` messages in the
|
||||
:class:`Bind` message. When that actor terminates the listen port will be
|
||||
closed and all resources associated with it will be released; existing
|
||||
connections will not be terminated at this point.
|
||||
|
||||
Closing connections
|
||||
^^^^^^^^^^^^^^^^^^^
|
||||
|
|
@ -478,8 +467,8 @@ actor.
|
|||
the remote endpoint. Pending writes will be flushed. If the close is successful, the listener will be notified with
|
||||
``Closed``.
|
||||
|
||||
``ConfirmedClose`` will close the sending direction of the connection by sending a ``FIN`` message, but receives
|
||||
will continue until the remote endpoint closes the connection, too. Pending writes will be flushed. If the close is
|
||||
``ConfirmedClose`` will close the sending direction of the connection by sending a ``FIN`` message, but data
|
||||
will continue to be received until the remote endpoint closes the connection, too. Pending writes will be flushed. If the close is
|
||||
successful, the listener will be notified with ``ConfirmedClosed``.
|
||||
|
||||
``Abort`` will immediately terminate the connection by sending a ``RST`` message to the remote endpoint. Pending
|
||||
|
|
@ -492,13 +481,125 @@ it receives one of the above close commands.
|
|||
|
||||
``ErrorClosed`` will be sent to the listener whenever an error happened that forced the connection to be closed.
|
||||
|
||||
All close notifications are subclasses of ``ConnectionClosed`` so listeners who do not need fine-grained close events
|
||||
All close notifications are sub-types of ``ConnectionClosed`` so listeners who do not need fine-grained close events
|
||||
may handle all close events in the same way.
|
||||
|
||||
Throttling Reads and Writes
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
*This section is not yet ready. More coming soon*
|
||||
The basic model of the TCP connection actor is that it has no internal
|
||||
buffering (i.e. it can only process one write at a time, meaning it can buffer
|
||||
one write until it has been passed on to the O/S kernel in full). Congestion
|
||||
needs to be handled at the user level, for which there are three modes of
|
||||
operation:
|
||||
|
||||
* *ACK-based:* every :class:`Write` command carries an arbitrary object, and if
|
||||
this object is not ``Tcp.NoAck`` then it will be returned to the sender of
|
||||
the :class:`Write` upon successfully writing all contained data to the
|
||||
socket. If no other write is initiated before having received this
|
||||
acknowledgement then no failures can happen due to buffer overrun.
|
||||
|
||||
* *NACK-based:* every write which arrives while a previous write is not yet
|
||||
completed will be replied to with a :class:`CommandFailed` message containing
|
||||
the failed write. Just relying on this mechanism requires the implemented
|
||||
protocol to tolerate skipping writes (e.g. if each write is a valid message
|
||||
on its own and it is not required that all are delivered). This mode is
|
||||
enabled by setting the ``useResumeWriting`` flag to ``false`` within the
|
||||
:class:`Register` message during connection activation.
|
||||
|
||||
* *NACK-based with write suspending:* this mode is very similar to the
|
||||
NACK-based one, but once a single write has failed no further writes will
|
||||
succeed until a :class:`ResumeWriting` message is received. This message will
|
||||
be answered with a :class:`WritingResumed` message once the last accepted
|
||||
write has completed. If the actor driving the connection implements buffering
|
||||
and resends the NACK’ed messages after having awaited the
|
||||
:class:`WritingResumed` signal then every message is delivered exactly once
|
||||
to the network socket.
|
||||
|
||||
These models (with the exception of the second which is rather specialised) are
|
||||
demonstrated in complete examples below. The full and contiguous source is
|
||||
available `on github <@github@/akka-docs/rst/scala/code/io/EchoServer.scala>`_.
|
||||
|
||||
.. note::
|
||||
|
||||
It should be obvious that all these flow control schemes only work between
|
||||
one writer and one connection actor; as soon as multiple actors send write
|
||||
commands to a single connection no consistent result can be achieved.
|
||||
|
||||
ACK-Based Back-Pressure
|
||||
^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
For proper function of the following example it is important to configure the
|
||||
connection to remain half-open when the remote side closed its writing end:
|
||||
this allows the example :class:`EchoHandler` to write all outstanding data back
|
||||
to the client before fully closing the connection. This is enabled using a flag
|
||||
upon connection activation (observe the :class:`Register` message):
|
||||
|
||||
.. includecode:: code/docs/io/EchoServer.scala#echo-manager
|
||||
|
||||
With this preparation let us dive into the handler itself:
|
||||
|
||||
.. includecode:: code/docs/io/EchoServer.scala#simple-echo-handler
|
||||
:exclude: storage-omitted
|
||||
|
||||
The principle is simple: when having written a chunk always wait for the
|
||||
``Ack`` to come back before sending the next chunk. While waiting we switch
|
||||
behavior such that new incoming data are buffered. The helper functions used
|
||||
are a bit lengthy but not complicated:
|
||||
|
||||
.. includecode:: code/docs/io/EchoServer.scala#simple-helpers
|
||||
|
||||
The most interesting part is probably the last: an ``Ack`` removes the oldest
|
||||
data chunk from the buffer, and if that was the last chunk then we either close
|
||||
the connection (if the peer closed its half already) or return to the idle
|
||||
behavior; otherwise we just send the next buffered chunk and stay waiting for
|
||||
the next ``Ack``.
|
||||
|
||||
Back-pressure can be propagated also across the reading side back to the writer
|
||||
on the other end of the connection by sending the :class:`SuspendReading`
|
||||
command to the connection actor. This will lead to no data being read from the
|
||||
socket anymore (although this does happen after a delay because it takes some
|
||||
time until the connection actor processes this command, hence appropriate
|
||||
head-room in the buffer should be present), which in turn will lead to the O/S
|
||||
kernel buffer filling up on our end, then the TCP window mechanism will stop
|
||||
the remote side from writing, filling up its write buffer, until finally the
|
||||
writer on the other side cannot push any data into the socket anymore. This is
|
||||
how end-to-end back-pressure is realized across a TCP connection.
|
||||
|
||||
NACK-Based Back-Pressure with Write Suspending
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
.. includecode:: code/docs/io/EchoServer.scala#echo-handler
|
||||
:exclude: buffering,closing,storage-omitted
|
||||
|
||||
The principle here is to keep writing until a :class:`CommandFailed` is
|
||||
received, using acknowledgements only to prune the resend buffer. When a such a
|
||||
failure was received, transition into a different state for handling and handle
|
||||
resending of all queued data:
|
||||
|
||||
.. includecode:: code/docs/io/EchoServer.scala#buffering
|
||||
|
||||
It should be noted that all writes which are currently buffered have also been
|
||||
sent to the connection actor upon entering this state, which means that the
|
||||
:class:`ResumeWriting` message is enqueued after those writes, leading to the
|
||||
reception of all outstanding :class:`CommandFailre` messages (which are ignored
|
||||
in this state) before receiving the :class:`WritingResumed` signal. That latter
|
||||
message is sent by the connection actor only once the internally queued write
|
||||
has been fully completed, meaning that a subsequent write will not fail. This
|
||||
is exploited by the :class:`EchoHandler` to switch to an ACK-based approach for
|
||||
the first ten writes after a failure before resuming the optimistic
|
||||
write-through behavior.
|
||||
|
||||
.. includecode:: code/docs/io/EchoServer.scala#closing
|
||||
|
||||
Closing the connection while still sending all data is a bit more involved than
|
||||
in the ACK-based approach: the idea is to always send all outstanding messages
|
||||
and acknowledge all successful writes, and if a failure happens then switch
|
||||
behavior to await the :class:`WritingResumed` event and start over.
|
||||
|
||||
The helper functions are very similar to the ACK-based case:
|
||||
|
||||
.. includecode:: code/docs/io/EchoServer.scala#helpers
|
||||
|
||||
Using UDP
|
||||
---------
|
||||
|
|
@ -643,12 +744,6 @@ will always be the endpoint we originally connected to.
|
|||
check, while in the case of connection-based UDP the security check is cached after connect, thus writes do
|
||||
not suffer an additional performance penalty.
|
||||
|
||||
Throttling Reads and Writes
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
*This section is not yet ready. More coming soon*
|
||||
|
||||
|
||||
Architecture in-depth
|
||||
---------------------
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue