add BackpressureBuffer, see #3253
- also make a Write’s “ack” be a Tcp.Event (to suit pipelines) - add stress test for BackpressureBuffer - add it to SslTlsSupportSpec - add it to the docs
This commit is contained in:
parent
025a91ecc2
commit
ea5b79e562
15 changed files with 764 additions and 248 deletions
|
|
@ -14,6 +14,7 @@ import akka.event.Logging;
|
|||
import akka.event.LoggingAdapter;
|
||||
import akka.io.Tcp.CommandFailed;
|
||||
import akka.io.Tcp.ConnectionClosed;
|
||||
import akka.io.Tcp.Event;
|
||||
import akka.io.Tcp.Received;
|
||||
import akka.io.Tcp.Write;
|
||||
import akka.io.Tcp.WritingResumed;
|
||||
|
|
@ -33,6 +34,13 @@ public class EchoHandler extends UntypedActor {
|
|||
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;
|
||||
|
||||
private static class Ack implements Event {
|
||||
public final int ack;
|
||||
public Ack(int ack) {
|
||||
this.ack = ack;
|
||||
}
|
||||
}
|
||||
|
||||
public EchoHandler(ActorRef connection, InetSocketAddress remote) {
|
||||
this.connection = connection;
|
||||
|
|
@ -50,7 +58,7 @@ public class EchoHandler extends UntypedActor {
|
|||
public void apply(Object msg) throws Exception {
|
||||
if (msg instanceof Received) {
|
||||
final ByteString data = ((Received) msg).data();
|
||||
connection.tell(TcpMessage.write(data, currentOffset()), getSelf());
|
||||
connection.tell(TcpMessage.write(data, new Ack(currentOffset())), getSelf());
|
||||
buffer(data);
|
||||
|
||||
} else if (msg instanceof Integer) {
|
||||
|
|
@ -59,7 +67,7 @@ public class EchoHandler extends UntypedActor {
|
|||
} else if (msg instanceof CommandFailed) {
|
||||
final Write w = (Write) ((CommandFailed) msg).cmd();
|
||||
connection.tell(TcpMessage.resumeWriting(), getSelf());
|
||||
getContext().become(buffering((Integer) w.ack()));
|
||||
getContext().become(buffering((Ack) w.ack()));
|
||||
|
||||
} else if (msg instanceof ConnectionClosed) {
|
||||
final ConnectionClosed cl = (ConnectionClosed) msg;
|
||||
|
|
@ -75,7 +83,7 @@ public class EchoHandler extends UntypedActor {
|
|||
};
|
||||
|
||||
//#buffering
|
||||
protected Procedure<Object> buffering(final int nack) {
|
||||
protected Procedure<Object> buffering(final Ack nack) {
|
||||
return new Procedure<Object>() {
|
||||
|
||||
private int toAck = 10;
|
||||
|
|
@ -99,7 +107,7 @@ public class EchoHandler extends UntypedActor {
|
|||
final int ack = (Integer) msg;
|
||||
acknowledge(ack);
|
||||
|
||||
if (ack >= nack) {
|
||||
if (ack >= nack.ack) {
|
||||
// otherwise it was the ack of the last successful write
|
||||
|
||||
if (storage.isEmpty()) {
|
||||
|
|
@ -216,12 +224,12 @@ public class EchoHandler extends UntypedActor {
|
|||
protected void writeAll() {
|
||||
int i = 0;
|
||||
for (ByteString data : storage) {
|
||||
connection.tell(TcpMessage.write(data, storageOffset + i++), getSelf());
|
||||
connection.tell(TcpMessage.write(data, new Ack(storageOffset + i++)), getSelf());
|
||||
}
|
||||
}
|
||||
|
||||
protected void writeFirst() {
|
||||
connection.tell(TcpMessage.write(storage.peek(), storageOffset), getSelf());
|
||||
connection.tell(TcpMessage.write(storage.peek(), new Ack(storageOffset)), getSelf());
|
||||
}
|
||||
|
||||
//#storage-omitted
|
||||
|
|
|
|||
|
|
@ -13,6 +13,7 @@ import akka.actor.UntypedActor;
|
|||
import akka.event.Logging;
|
||||
import akka.event.LoggingAdapter;
|
||||
import akka.io.Tcp.ConnectionClosed;
|
||||
import akka.io.Tcp.Event;
|
||||
import akka.io.Tcp.Received;
|
||||
import akka.io.TcpMessage;
|
||||
import akka.japi.Procedure;
|
||||
|
|
@ -85,7 +86,7 @@ public class SimpleEchoHandler extends UntypedActor {
|
|||
private boolean suspended = false;
|
||||
private boolean closing = false;
|
||||
|
||||
private final Object ACK = new Object();
|
||||
private final Event ACK = new Event() {};
|
||||
|
||||
//#simple-helpers
|
||||
protected void buffer(ByteString data) {
|
||||
|
|
|
|||
|
|
@ -21,8 +21,13 @@ import akka.actor.UntypedActor;
|
|||
import akka.event.Logging;
|
||||
import akka.event.LoggingAdapter;
|
||||
import akka.io.AbstractPipelineContext;
|
||||
import akka.io.BackpressureBuffer;
|
||||
import akka.io.DelimiterFraming;
|
||||
import akka.io.HasLogging;
|
||||
import akka.io.PipelineStage;
|
||||
import static akka.io.PipelineStage.sequence;
|
||||
import akka.io.SslTlsSupport;
|
||||
import akka.io.StringByteStringAdapter;
|
||||
import akka.io.Tcp;
|
||||
import akka.io.Tcp.Bound;
|
||||
import akka.io.Tcp.Command;
|
||||
|
|
@ -33,6 +38,8 @@ import akka.io.Tcp.Received;
|
|||
import akka.io.TcpMessage;
|
||||
import akka.io.TcpPipelineHandler;
|
||||
import akka.io.TcpPipelineHandler.Init;
|
||||
import akka.io.TcpPipelineHandler.WithinActorContext;
|
||||
import akka.io.TcpReadWriteAdapter;
|
||||
import akka.io.ssl.SslTlsSupportSpec;
|
||||
import akka.testkit.AkkaSpec;
|
||||
import akka.testkit.JavaTestKit;
|
||||
|
|
@ -60,14 +67,8 @@ public class SslDocTest {
|
|||
.tell(TcpMessage.connect(remote), getSelf());
|
||||
}
|
||||
|
||||
class Context extends AbstractPipelineContext implements HasLogging {
|
||||
@Override
|
||||
public LoggingAdapter getLogger() {
|
||||
return log;
|
||||
}
|
||||
}
|
||||
|
||||
Init<HasLogging, Command, Event> init = null;
|
||||
// this will hold the pipeline handler’s context
|
||||
Init<WithinActorContext, String, String> init = null;
|
||||
|
||||
@Override
|
||||
public void onReceive(Object msg) {
|
||||
|
|
@ -79,33 +80,30 @@ public class SslDocTest {
|
|||
final SSLEngine engine = sslContext.createSSLEngine(
|
||||
remote.getHostName(), remote.getPort());
|
||||
engine.setUseClientMode(true);
|
||||
final SslTlsSupport ssl = new SslTlsSupport(engine);
|
||||
|
||||
// set up the context for communicating with TcpPipelineHandler
|
||||
init = new Init<HasLogging, Command, Event>(ssl) {
|
||||
@Override
|
||||
public HasLogging makeContext(ActorContext ctx) {
|
||||
return new Context();
|
||||
}
|
||||
};
|
||||
// build pipeline and set up context for communicating with TcpPipelineHandler
|
||||
init = TcpPipelineHandler.withLogger(log, sequence(sequence(sequence(sequence(
|
||||
new StringByteStringAdapter("utf-8"),
|
||||
new DelimiterFraming(1024, ByteString.fromString("\n"), true)),
|
||||
new TcpReadWriteAdapter()),
|
||||
new SslTlsSupport(engine)),
|
||||
new BackpressureBuffer(1000, 10000, 1000000)));
|
||||
|
||||
// create handler for pipeline, setting ourselves as payload recipient
|
||||
final ActorRef handler = getContext().actorOf(
|
||||
TcpPipelineHandler.create(init, getSender(), getSelf()));
|
||||
|
||||
// register the SSL handler with the connection
|
||||
getSender().tell(TcpMessage.register(handler), getSelf());
|
||||
|
||||
// and send a message across the SSL channel
|
||||
handler.tell(
|
||||
init.command(TcpMessage.write(ByteString.fromString("hello"))),
|
||||
getSelf());
|
||||
handler.tell(init.command("hello\n"), getSelf());
|
||||
|
||||
} else if (msg instanceof Init.Event) {
|
||||
// unwrap TcpPipelineHandler’s event into a Tcp.Event
|
||||
final Event recv = init.event(msg);
|
||||
if (recv instanceof Received) {
|
||||
// and inform someone of the received payload
|
||||
listener.tell(((Received) recv).data().utf8String(), getSelf());
|
||||
}
|
||||
final String recv = init.event(msg);
|
||||
// and inform someone of the received payload
|
||||
listener.tell(recv, getSelf());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -130,14 +128,8 @@ public class SslDocTest {
|
|||
getSelf());
|
||||
}
|
||||
|
||||
class Context extends AbstractPipelineContext implements HasLogging {
|
||||
@Override
|
||||
public LoggingAdapter getLogger() {
|
||||
return log;
|
||||
}
|
||||
}
|
||||
|
||||
Init<HasLogging, Command, Event> init = null;
|
||||
// this will hold the pipeline handler’s context
|
||||
Init<WithinActorContext, String, String> init = null;
|
||||
|
||||
@Override
|
||||
public void onReceive(Object msg) {
|
||||
|
|
@ -153,15 +145,15 @@ public class SslDocTest {
|
|||
final SSLEngine engine = sslContext.createSSLEngine(
|
||||
remote.getHostName(), remote.getPort());
|
||||
engine.setUseClientMode(false);
|
||||
final SslTlsSupport ssl = new SslTlsSupport(engine);
|
||||
|
||||
// set up the context for communicating with TcpPipelineHandler
|
||||
init = new Init<HasLogging, Command, Event>(ssl) {
|
||||
@Override
|
||||
public HasLogging makeContext(ActorContext ctx) {
|
||||
return new Context();
|
||||
}
|
||||
};
|
||||
|
||||
// build pipeline and set up context for communicating with TcpPipelineHandler
|
||||
init = TcpPipelineHandler.withLogger(log, sequence(sequence(sequence(sequence(
|
||||
new StringByteStringAdapter("utf-8"),
|
||||
new DelimiterFraming(1024, ByteString.fromString("\n"), true)),
|
||||
new TcpReadWriteAdapter()),
|
||||
new SslTlsSupport(engine)),
|
||||
new BackpressureBuffer(1000, 10000, 1000000)));
|
||||
|
||||
// create handler for pipeline, setting ourselves as payload recipient
|
||||
final ActorRef handler = getContext().actorOf(
|
||||
TcpPipelineHandler.create(init, getSender(), getSelf()));
|
||||
|
|
@ -171,14 +163,11 @@ public class SslDocTest {
|
|||
|
||||
} else if (msg instanceof Init.Event) {
|
||||
// unwrap TcpPipelineHandler’s event to get a Tcp.Event
|
||||
final Event recv = init.event(msg);
|
||||
if (recv instanceof Received) {
|
||||
// inform someone of the received message
|
||||
listener.tell(((Received) recv).data().utf8String(), getSelf());
|
||||
// and reply (sender is the SSL handler created above)
|
||||
getSender().tell(init.command(
|
||||
TcpMessage.write(ByteString.fromString("world"))), getSelf());
|
||||
}
|
||||
final String recv = init.event(msg);
|
||||
// inform someone of the received message
|
||||
listener.tell(recv, getSelf());
|
||||
// and reply (sender is the SSL handler created above)
|
||||
getSender().tell(init.command("world\n"), getSelf());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -201,9 +190,9 @@ public class SslDocTest {
|
|||
assert getLastSender() == server;
|
||||
|
||||
final ActorRef client = system.actorOf(Props.create(SslClient.class, bound.localAddress(), ctx, getRef()));
|
||||
expectMsgEquals("hello");
|
||||
expectMsgEquals("hello\n");
|
||||
assert getLastSender() == server;
|
||||
expectMsgEquals("world");
|
||||
expectMsgEquals("world\n");
|
||||
assert getLastSender() == client;
|
||||
}
|
||||
};
|
||||
|
|
|
|||
|
|
@ -563,7 +563,7 @@ resending of all queued data:
|
|||
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
|
||||
reception of all outstanding :class:`CommandFailed` 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
|
||||
|
|
@ -590,36 +590,55 @@ first look at the SSL server:
|
|||
|
||||
.. includecode:: code/docs/io/japi/SslDocTest.java#server
|
||||
|
||||
Please refer to `the source code`_ to see all imports.
|
||||
|
||||
.. _the source code: @github@/akka-docs/rst/java/code/docs/io/japi/SslDocTest.java
|
||||
|
||||
The actor above binds to a local port and registers itself as the handler for
|
||||
new connections. When a new connection comes in it will create a
|
||||
:class:`javax.net.ssl.SSLEngine` (details not shown here since they vary wildly
|
||||
:class:`javax.net.ssl.SSLEngine` (details not shown here since they vary widely
|
||||
for different setups, please refer to the JDK documentation) and wrap that in
|
||||
an :class:`SslTlsSupport` pipeline stage (which is included in ``akka-actor``).
|
||||
This single-stage pipeline will be driven by a :class:`TcpPipelineHandler`
|
||||
actor which is also included in ``akka-actor``. In order to capture the generic
|
||||
command and event types consumed and emitted by that actor we need to create a
|
||||
wrapper—the nested :class:`Init` class—which also provides the
|
||||
:meth:`makeContext` method for creating the pipeline context needed by the
|
||||
supplied pipeline. With those things bundled up all that remains is creating a
|
||||
|
||||
This sample demonstrates a few more things: below the SSL pipeline stage we
|
||||
have inserted a backpressure buffer which will generate a
|
||||
:class:`HighWatermarkReached` event to tell the upper stages to suspend writing
|
||||
(generated at 10000 buffered bytes) and a :class:`LowWatermarkReached` when
|
||||
they can resume writing (when buffer empties below 1000 bytes); the buffer has
|
||||
a maximum capacity of 1MB. The implementation is very similar to the NACK-based
|
||||
backpressure approach presented above. Above the SSL stage comes an adapter
|
||||
which extracts only the payload data from the TCP commands and events, i.e. it
|
||||
speaks :class:`ByteString` above. The resulting byte streams are broken into
|
||||
frames by a :class:`DelimiterFraming` stage which chops them up on newline
|
||||
characters. The top-most stage then converts between :class:`String` and UTF-8
|
||||
encoded :class:`ByteString`.
|
||||
|
||||
As a result the pipeline will accept simple :class:`String` commands, encode
|
||||
them using UTF-8, delimit them with newlines (which are expected to be already
|
||||
present in the sending direction), transform them into TCP commands and events,
|
||||
encrypt them and send them off to the connection actor while buffering writes.
|
||||
|
||||
This pipeline is driven by a :class:`TcpPipelineHandler` actor which is also
|
||||
included in ``akka-actor``. In order to capture the generic command and event
|
||||
types consumed and emitted by that actor we need to create a wrapper—the nested
|
||||
:class:`Init` class—which also provides the the pipeline context needed by the
|
||||
supplied pipeline; in this case we use the :meth:`withLogger` convenience
|
||||
method which supplies a context that implements :class:`HasLogger` and
|
||||
:class:`HasActorContext` and should be sufficient for typical pipelines. With
|
||||
those things bundled up all that remains is creating a
|
||||
:class:`TcpPipelineHandler` and registering that one as the recipient of
|
||||
inbound traffic from the TCP connection.
|
||||
|
||||
Since we instructed that handler actor to send any events which are emitted by
|
||||
the SSL pipeline to ourselves, we can then just wait for the reception of the
|
||||
decrypted payload messages, compute a response—just ``"world"`` in this
|
||||
case—and reply by sending back a ``Tcp.Write``. It should be noted that
|
||||
decrypted payload messages, compute a response—just ``"world\n"`` in this
|
||||
case—and reply by sending back an ``Init.Command``. It should be noted that
|
||||
communication with the handler wraps commands and events in the inner types of
|
||||
the ``init`` object in order to keep things well separated. To ease handling of
|
||||
such path-dependent types there exist two helper methods, namely
|
||||
:class:`Init.command` for creating a command and :class:`Init.event` for
|
||||
unwrapping an event.
|
||||
|
||||
.. warning::
|
||||
|
||||
The :class:`TcpPipelineHandler` does currently not handle back-pressure from
|
||||
the TCP socket, i.e. it will just lose data when the kernel buffer
|
||||
overflows. This will be fixed before Akka 2.2 final.
|
||||
|
||||
Looking at the client side we see that not much needs to be changed:
|
||||
|
||||
.. includecode:: code/docs/io/japi/SslDocTest.java#client
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue