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:
Roland 2013-04-16 22:31:09 +02:00
parent 489c00b913
commit 0e34edbcb3
20 changed files with 1874 additions and 187 deletions

View file

@ -0,0 +1,64 @@
/**
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.actor;
import java.util.concurrent.TimeUnit;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.concurrent.duration.Duration;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.actor.Inbox;
import akka.actor.PoisonPill;
import akka.actor.Terminated;
import akka.testkit.AkkaSpec;
import akka.testkit.JavaTestKit;
public class InboxDocTest {
private static ActorSystem system;
@BeforeClass
public static void beforeAll() {
system = ActorSystem.create("MySystem", AkkaSpec.testConf());
}
@AfterClass
public static void afterAll() {
system.shutdown();
system = null;
}
@Test
public void demonstrateInbox() {
final JavaTestKit probe = new JavaTestKit(system);
final ActorRef target = probe.getRef();
//#inbox
final Inbox inbox = Inbox.create(system);
inbox.send(target, "hello");
//#inbox
probe.expectMsgEquals("hello");
probe.send(probe.getLastSender(), "world");
//#inbox
assert inbox.receive(Duration.create(1, TimeUnit.SECONDS)).equals("world");
//#inbox
}
@Test
public void demonstrateWatch() {
final JavaTestKit probe = new JavaTestKit(system);
final ActorRef target = probe.getRef();
//#watch
final Inbox inbox = Inbox.create(system);
inbox.watch(target);
target.tell(PoisonPill.getInstance(), null);
assert inbox.receive(Duration.create(1, TimeUnit.SECONDS)) instanceof Terminated;
//#watch
}
}

View file

@ -0,0 +1,229 @@
/**
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.io.japi;
import java.net.InetSocketAddress;
import java.util.LinkedList;
import java.util.Queue;
import akka.actor.ActorRef;
import akka.actor.UntypedActor;
import akka.event.Logging;
import akka.event.LoggingAdapter;
import akka.io.Tcp.CommandFailed;
import akka.io.Tcp.ConnectionClosed;
import akka.io.Tcp.Received;
import akka.io.Tcp.Write;
import akka.io.Tcp.WritingResumed;
import akka.io.TcpMessage;
import akka.japi.Procedure;
import akka.util.ByteString;
//#echo-handler
public class EchoHandler extends UntypedActor {
final LoggingAdapter log = Logging
.getLogger(getContext().system(), getSelf());
final ActorRef connection;
final InetSocketAddress remote;
public static final long MAX_STORED = 100000000;
public static final long HIGH_WATERMARK = MAX_STORED * 5 / 10;
public static final long LOW_WATERMARK = MAX_STORED * 2 / 10;
public EchoHandler(ActorRef connection, InetSocketAddress remote) {
this.connection = connection;
this.remote = remote;
// sign death pact: this actor stops when the connection is closed
getContext().watch(connection);
// start out in optimistic write-through mode
getContext().become(writing);
}
private final Procedure<Object> writing = new Procedure<Object>() {
@Override
public void apply(Object msg) throws Exception {
if (msg instanceof Received) {
final ByteString data = ((Received) msg).data();
connection.tell(TcpMessage.write(data, currentOffset()), getSelf());
buffer(data);
} else if (msg instanceof Integer) {
acknowledge((Integer) msg);
} else if (msg instanceof CommandFailed) {
final Write w = (Write) ((CommandFailed) msg).cmd();
connection.tell(TcpMessage.resumeWriting(), getSelf());
getContext().become(buffering((Integer) w.ack()));
} else if (msg instanceof ConnectionClosed) {
final ConnectionClosed cl = (ConnectionClosed) msg;
if (cl.isPeerClosed()) {
if (storage.isEmpty()) {
getContext().stop(getSelf());
} else {
getContext().become(closing);
}
}
}
}
};
//#buffering
protected Procedure<Object> buffering(final int nack) {
return new Procedure<Object>() {
private int toAck = 10;
private boolean peerClosed = false;
@Override
public void apply(Object msg) throws Exception {
if (msg instanceof Received) {
buffer(((Received) msg).data());
} else if (msg instanceof WritingResumed) {
writeFirst();
} else if (msg instanceof ConnectionClosed) {
if (((ConnectionClosed) msg).isPeerClosed())
peerClosed = true;
else
getContext().stop(getSelf());
} else if (msg instanceof Integer) {
final int ack = (Integer) msg;
acknowledge(ack);
if (ack >= nack) {
// otherwise it was the ack of the last successful write
if (storage.isEmpty()) {
if (peerClosed)
getContext().stop(getSelf());
else
getContext().become(writing);
} else {
if (toAck > 0) {
// stay in ACK-based mode for a short while
writeFirst();
--toAck;
} else {
// then return to NACK-based again
writeAll();
if (peerClosed)
getContext().become(closing);
else
getContext().become(writing);
}
}
}
}
}
};
}
//#buffering
//#closing
protected Procedure<Object> closing = new Procedure<Object>() {
@Override
public void apply(Object msg) throws Exception {
if (msg instanceof CommandFailed) {
// the command can only have been a Write
connection.tell(TcpMessage.resumeWriting(), getSelf());
getContext().become(closeResend, false);
} else if (msg instanceof Integer) {
acknowledge((Integer) msg);
if (storage.isEmpty())
getContext().stop(getSelf());
}
}
};
protected Procedure<Object> closeResend = new Procedure<Object>() {
@Override
public void apply(Object msg) throws Exception {
if (msg instanceof WritingResumed) {
writeAll();
getContext().unbecome();
} else if (msg instanceof Integer) {
acknowledge((Integer) msg);
}
}
};
//#closing
//#storage-omitted
@Override
public void onReceive(Object msg) throws Exception {
// this method is not used due to become()
}
@Override
public void postStop() {
log.info("transferred {} bytes from/to [{}]", transferred, remote);
}
private long transferred;
private int storageOffset = 0;
private long stored = 0;
private Queue<ByteString> storage = new LinkedList<ByteString>();
private boolean suspended = false;
//#helpers
protected void buffer(ByteString data) {
storage.add(data);
stored += data.size();
if (stored > MAX_STORED) {
log.warning("drop connection to [{}] (buffer overrun)", remote);
getContext().stop(getSelf());
} else if (stored > HIGH_WATERMARK) {
log.debug("suspending reading at {}", currentOffset());
connection.tell(TcpMessage.suspendReading(), getSelf());
suspended = true;
}
}
protected void acknowledge(int ack) {
assert ack == storageOffset;
assert !storage.isEmpty();
final ByteString acked = storage.remove();
stored -= acked.size();
transferred += acked.size();
storageOffset += 1;
if (suspended && stored < LOW_WATERMARK) {
log.debug("resuming reading");
connection.tell(TcpMessage.resumeReading(), getSelf());
suspended = false;
}
}
//#helpers
protected int currentOffset() {
return storageOffset + storage.size();
}
protected void writeAll() {
int i = 0;
for (ByteString data : storage) {
connection.tell(TcpMessage.write(data, storageOffset + i++), getSelf());
}
}
protected void writeFirst() {
connection.tell(TcpMessage.write(storage.peek(), storageOffset), getSelf());
}
//#storage-omitted
}
//#echo-handler

View file

@ -0,0 +1,81 @@
/**
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.io.japi;
import java.net.InetSocketAddress;
import akka.actor.ActorRef;
import akka.actor.Props;
import akka.actor.SupervisorStrategy;
import akka.actor.UntypedActor;
import akka.event.Logging;
import akka.event.LoggingAdapter;
import akka.io.Tcp;
import akka.io.Tcp.Bind;
import akka.io.Tcp.Bound;
import akka.io.Tcp.CommandFailed;
import akka.io.Tcp.Connected;
import akka.io.TcpMessage;
public class EchoManager extends UntypedActor {
final LoggingAdapter log = Logging
.getLogger(getContext().system(), getSelf());
final Class<?> handlerClass;
public EchoManager(Class<?> handlerClass) {
this.handlerClass = handlerClass;
}
@Override
public SupervisorStrategy supervisorStrategy() {
return SupervisorStrategy.stoppingStrategy();
}
@Override
public void preStart() throws Exception {
//#manager
final ActorRef tcpManager = Tcp.get(getContext().system()).manager();
//#manager
tcpManager.tell(
TcpMessage.bind(getSelf(), new InetSocketAddress("localhost", 0), 100),
getSelf());
}
@Override
public void postRestart(Throwable arg0) throws Exception {
// do not restart
getContext().stop(getSelf());
}
@Override
public void onReceive(Object msg) throws Exception {
if (msg instanceof Bound) {
log.info("listening on [{}]", ((Bound) msg).localAddress());
} else if (msg instanceof Tcp.CommandFailed) {
final CommandFailed failed = (CommandFailed) msg;
if (failed.cmd() instanceof Bind) {
log.warning("cannot bind to [{}]", ((Bind) failed.cmd()).endpoint());
getContext().stop(getSelf());
} else {
log.warning("unknown command failed [{}]", failed.cmd());
}
} else
if (msg instanceof Connected) {
final Connected conn = (Connected) msg;
log.info("received connection from [{}]", conn.remoteAddress());
final ActorRef connection = getSender();
final ActorRef handler = getContext().actorOf(
Props.create(handlerClass, connection, conn.remoteAddress()));
//#echo-manager
connection.tell(TcpMessage.register(handler,
true, // <-- keepOpenOnPeerClosed flag
true), getSelf());
//#echo-manager
}
}
}

View file

@ -0,0 +1,35 @@
/**
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.io.japi;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.actor.Props;
import com.typesafe.config.Config;
import com.typesafe.config.ConfigFactory;
public class EchoServer {
public static void main(String[] args) throws InterruptedException {
final Config config = ConfigFactory.parseString("akka.loglevel=DEBUG");
final ActorSystem system = ActorSystem.create("EchoServer", config);
try {
final CountDownLatch latch = new CountDownLatch(1);
final ActorRef watcher = system.actorOf(Props.create(Watcher.class, latch), "watcher");
final ActorRef nackServer = system.actorOf(Props.create(EchoManager.class, EchoHandler.class), "nack");
final ActorRef ackServer = system.actorOf(Props.create(EchoManager.class, SimpleEchoHandler.class), "ack");
watcher.tell(nackServer, null);
watcher.tell(ackServer, null);
latch.await(10, TimeUnit.MINUTES);
} finally {
system.shutdown();
}
}
}

View file

@ -0,0 +1,179 @@
/**
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.io.japi;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
//#imports
import java.net.InetSocketAddress;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.actor.Props;
import akka.actor.UntypedActor;
import akka.io.Tcp;
import akka.io.Tcp.Bound;
import akka.io.Tcp.CommandFailed;
import akka.io.Tcp.Connected;
import akka.io.Tcp.ConnectionClosed;
import akka.io.Tcp.Received;
import akka.io.TcpMessage;
import akka.japi.Procedure;
import akka.util.ByteString;
//#imports
import akka.testkit.JavaTestKit;
import akka.testkit.AkkaSpec;
public class IODocTest {
static
//#server
public class Server extends UntypedActor {
final ActorRef manager;
public Server(ActorRef manager) {
this.manager = manager;
}
@Override
public void preStart() throws Exception {
final ActorRef tcp = Tcp.get(getContext().system()).manager();
tcp.tell(TcpMessage.bind(getSelf(),
new InetSocketAddress("localhost", 0), 100), getSelf());
}
@Override
public void onReceive(Object msg) throws Exception {
if (msg instanceof Bound) {
manager.tell(msg, getSelf());
} else if (msg instanceof CommandFailed) {
getContext().stop(getSelf());
} else if (msg instanceof Connected) {
final Connected conn = (Connected) msg;
manager.tell(conn, getSelf());
final ActorRef handler = getContext().actorOf(
Props.create(SimplisticHandler.class));
getSender().tell(TcpMessage.register(handler), getSelf());
}
}
}
//#server
static
//#simplistic-handler
public class SimplisticHandler extends UntypedActor {
@Override
public void onReceive(Object msg) throws Exception {
if (msg instanceof Received) {
final ByteString data = ((Received) msg).data();
System.out.println(data);
getSender().tell(TcpMessage.write(data), getSelf());
} else if (msg instanceof ConnectionClosed) {
getContext().stop(getSelf());
}
}
}
//#simplistic-handler
static
//#client
public class Client extends UntypedActor {
final InetSocketAddress remote;
final ActorRef listener;
public Client(InetSocketAddress remote, ActorRef listener) {
this.remote = remote;
this.listener = listener;
final ActorRef tcp = Tcp.get(getContext().system()).manager();
tcp.tell(TcpMessage.connect(remote), getSelf());
}
@Override
public void onReceive(Object msg) throws Exception {
if (msg instanceof CommandFailed) {
listener.tell("failed", getSelf());
getContext().stop(getSelf());
} else if (msg instanceof Connected) {
listener.tell(msg, getSelf());
getSender().tell(TcpMessage.register(getSelf()), getSelf());
getContext().become(connected(getSender()));
}
}
private Procedure<Object> connected(final ActorRef connection) {
return new Procedure<Object>() {
@Override
public void apply(Object msg) throws Exception {
if (msg instanceof ByteString) {
connection.tell(TcpMessage.write((ByteString) msg), getSelf());
} else if (msg instanceof CommandFailed) {
// OS kernel socket buffer was full
} else if (msg instanceof Received) {
listener.tell(((Received) msg).data(), getSelf());
} else if (msg.equals("close")) {
connection.tell(TcpMessage.close(), getSelf());
} else if (msg instanceof ConnectionClosed) {
getContext().stop(getSelf());
}
}
};
}
}
//#client
private static ActorSystem system;
@BeforeClass
public static void setup() {
system = ActorSystem.create("IODocTest", AkkaSpec.testConf());
}
@AfterClass
public static void teardown() {
system.shutdown();
}
@Test
public void testConnection() {
new JavaTestKit(system) {
{
@SuppressWarnings("unused")
final ActorRef server = system.actorOf(Props.create(Server.class, getRef()), "server1");
final InetSocketAddress listen = expectMsgClass(Bound.class).localAddress();
final ActorRef client = system.actorOf(Props.create(Client.class, listen, getRef()), "client1");
final Connected c1 = expectMsgClass(Connected.class);
final Connected c2 = expectMsgClass(Connected.class);
assert c1.localAddress().equals(c2.remoteAddress());
assert c2.localAddress().equals(c1.remoteAddress());
client.tell(ByteString.fromString("hello"), getRef());
final ByteString reply = expectMsgClass(ByteString.class);
assert reply.utf8String().equals("hello");
watch(client);
client.tell("close", getRef());
expectTerminated(client);
}
};
}
}

View file

@ -0,0 +1,130 @@
/**
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.io.japi;
import java.net.InetSocketAddress;
import java.util.LinkedList;
import java.util.Queue;
import akka.actor.ActorRef;
import akka.actor.UntypedActor;
import akka.event.Logging;
import akka.event.LoggingAdapter;
import akka.io.Tcp.ConnectionClosed;
import akka.io.Tcp.Received;
import akka.io.TcpMessage;
import akka.japi.Procedure;
import akka.util.ByteString;
//#simple-echo-handler
public class SimpleEchoHandler extends UntypedActor {
final LoggingAdapter log = Logging
.getLogger(getContext().system(), getSelf());
final ActorRef connection;
final InetSocketAddress remote;
public static final long maxStored = 100000000;
public static final long highWatermark = maxStored * 5 / 10;
public static final long lowWatermark = maxStored * 2 / 10;
public SimpleEchoHandler(ActorRef connection, InetSocketAddress remote) {
this.connection = connection;
this.remote = remote;
// sign death pact: this actor stops when the connection is closed
getContext().watch(connection);
}
@Override
public void onReceive(Object msg) throws Exception {
if (msg instanceof Received) {
final ByteString data = ((Received) msg).data();
buffer(data);
connection.tell(TcpMessage.write(data, ACK), getSelf());
// now switch behavior to waiting for acknowledgement
getContext().become(buffering, false);
} else if (msg instanceof ConnectionClosed) {
getContext().stop(getSelf());
}
}
private final Procedure<Object> buffering = new Procedure<Object>() {
@Override
public void apply(Object msg) throws Exception {
if (msg instanceof Received) {
buffer(((Received) msg).data());
} else if (msg == ACK) {
acknowledge();
} else if (msg instanceof ConnectionClosed) {
if (((ConnectionClosed) msg).isPeerClosed()) {
closing = true;
} else {
// could also be ErrorClosed, in which case we just give up
getContext().stop(getSelf());
}
}
}
};
//#storage-omitted
public void postStop() {
log.info("transferred {} bytes from/to [{}]", transferred, remote);
}
private long transferred;
private long stored = 0;
private Queue<ByteString> storage = new LinkedList<ByteString>();
private boolean suspended = false;
private boolean closing = false;
private final Object ACK = new Object();
//#simple-helpers
protected void buffer(ByteString data) {
storage.add(data);
stored += data.size();
if (stored > maxStored) {
log.warning("drop connection to [{}] (buffer overrun)", remote);
getContext().stop(getSelf());
} else if (stored > highWatermark) {
log.debug("suspending reading");
connection.tell(TcpMessage.suspendReading(), getSelf());
suspended = true;
}
}
protected void acknowledge() {
final ByteString acked = storage.remove();
stored -= acked.size();
transferred += acked.size();
if (suspended && stored < lowWatermark) {
log.debug("resuming reading");
connection.tell(TcpMessage.resumeReading(), getSelf());
suspended = false;
}
if (storage.isEmpty()) {
if (closing) {
getContext().stop(getSelf());
} else {
getContext().unbecome();
}
} else {
connection.tell(TcpMessage.write(storage.peek(), ACK), getSelf());
}
}
//#simple-helpers
//#storage-omitted
}
//#simple-echo-handler

View file

@ -0,0 +1,34 @@
package docs.io.japi;
import java.util.concurrent.CountDownLatch;
import akka.actor.ActorRef;
import akka.actor.Terminated;
import akka.actor.UntypedActor;
public class Watcher extends UntypedActor {
static public class Watch {
final ActorRef target;
public Watch(ActorRef target) {
this.target = target;
}
}
final CountDownLatch latch;
public Watcher(CountDownLatch latch) {
this.latch = latch;
}
@Override
public void onReceive(Object msg) throws Exception {
if (msg instanceof Watch) {
getContext().watch(((Watch) msg).target);
} else if (msg instanceof Terminated) {
latch.countDown();
if (latch.getCount() == 0) getContext().stop(getSelf());
}
}
}

View file

@ -11,7 +11,12 @@ 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.
Terminology, Concepts
---------------------
@ -21,7 +26,7 @@ as an entry point for the API. I/O is broken into several drivers. The manager f
is accessible by querying an ``ActorSystem``. For example the following code
looks up the TCP manager and returns its ``ActorRef``:
.. includecode:: code/docs/io/IODocTest.java#manager
.. includecode:: code/docs/io/japi/EchoManager.java#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
@ -346,84 +351,92 @@ this must be modeled either as a command or event, i.e. it will be part of the
Using TCP
---------
The following imports are assumed throughout this section:
The code snippets through-out this section assume the following imports:
.. includecode:: code/docs/io/IODocTest.java#imports
.. includecode:: code/docs/io/japi/IODocTest.java#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.
.. includecode:: code/docs/io/IODocTest.java#manager
.. includecode:: code/docs/io/japi/EchoManager.java#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-java:
Connecting
^^^^^^^^^^
The first step of connecting to a remote address is sending a ``Connect`` message to the TCP manager:
.. includecode:: code/docs/io/japi/IODocTest.java#client
.. includecode:: code/docs/io/IODocTest.java#connect
When connecting, it is also possible to set various socket options or specify a local address:
.. includecode:: code/docs/io/IODocTest.java#connect-with-options
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.
.. includecode:: code/docs/io/IODocTest.java#connected
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.
When receiving the :class:`Connected` message 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, which therefore done in the last line above.
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.
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:
.. includecode:: code/docs/io/IODocTest.java#received
``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-java` 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 connection, 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/japi/IODocTest.java#server
.. includecode:: code/docs/io/IODocTest.java#bind
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.
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-java>`: when an incoming connection is established, the actor provided as ``handler`` will
receive a ``Connected`` message whose sender is the connection actor.
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).
.. includecode:: code/docs/io/IODocTest.java#connected
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:
When receiving the :class:`Connected` message 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, which therefore done in the last line above.
.. includecode:: code/docs/io/japi/IODocTest.java#simplistic-handler
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 released. During the
connection lifetime the listener will receive various event notifications in the same way as in the outbound
connection case.
For a more complete sample which also takes into account the possibility of
failures when sending please see `Throttling Reads and Writes`_ below.
.. _closing-connections-java:
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
^^^^^^^^^^^^^^^^^^^
@ -435,8 +448,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
@ -449,13 +462,126 @@ 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 NACKed 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/java/code/io/japi>`_.
.. 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/japi/EchoManager.java#echo-manager
With this preparation let us dive into the handler itself:
.. includecode:: code/docs/io/japi/SimpleEchoHandler.java#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/japi/SimpleEchoHandler.java#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/japi/EchoHandler.java#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/japi/EchoHandler.java#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/japi/EchoHandler.java#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/japi/EchoHandler.java#helpers
Using UDP
---------
@ -568,12 +694,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 does
not suffer an additional performance penalty.
Throttling Reads and Writes
^^^^^^^^^^^^^^^^^^^^^^^^^^^
*This section is not yet ready. More coming soon*
Architecture in-depth
---------------------

View file

@ -168,6 +168,23 @@ constructor arguments are determined by a dependency injection framework.
When using a dependency injection framework, actor beans *MUST NOT* have
singleton scope.
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:: code/docs/actor/InboxDocTest.java#inbox
The :meth:`send` method wraps a normal :meth:`tell` and supplies the internal
actors reference as the sender. This allows the reply to be received on the
last line. Watching an actor is quite simple as well:
.. includecode:: code/docs/actor/InboxDocTest.java#watch
UntypedActor API
================