Merge pull request #1291 from akka/wip-3174-pipelines-∂π

implement and document Pipelines, see #3174
This commit is contained in:
Roland Kuhn 2013-04-08 13:13:23 -07:00
commit 2375972969
21 changed files with 2540 additions and 65 deletions

View file

@ -33,7 +33,8 @@ public class InitializationDocSpecJava {
// of the actor. To opt-out from stopping the children, we
// have to override preRestart()
@Override
public void preRestart(Throwable reason, Option<Object> message) {
public void preRestart(Throwable reason, Option<Object> message)
throws Exception {
// Keep the call to postStop(), but no stopping of children
postStop();
}

View file

@ -0,0 +1,16 @@
/**
* Copyright (C) 2013 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.io.japi;
import akka.actor.ActorContext;
import akka.io.PipelineContext;
//#actor-context
public interface HasActorContext extends PipelineContext {
public ActorContext getContext();
}
//#actor-context

View file

@ -0,0 +1,15 @@
/**
* Copyright (C) 2013 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.io.japi;
import java.nio.ByteOrder;
import akka.io.PipelineContext;
public interface HasByteOrder extends PipelineContext {
public ByteOrder byteOrder();
}

View file

@ -0,0 +1,84 @@
/**
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.io.japi;
//#frame
import java.nio.ByteOrder;
import java.util.ArrayList;
import scala.util.Either;
import akka.io.AbstractSymmetricPipePair;
import akka.io.PipePairFactory;
import akka.io.PipelineContext;
import akka.io.SymmetricPipePair;
import akka.io.SymmetricPipelineStage;
import akka.util.ByteString;
import akka.util.ByteStringBuilder;
public class LengthFieldFrame extends
SymmetricPipelineStage<PipelineContext, ByteString, ByteString> {
final int maxSize;
public LengthFieldFrame(int maxSize) {
this.maxSize = maxSize;
}
@Override
public SymmetricPipePair<ByteString, ByteString> apply(final PipelineContext ctx) {
return PipePairFactory
.create(ctx, new AbstractSymmetricPipePair<ByteString, ByteString>() {
final ByteOrder byteOrder = ByteOrder.BIG_ENDIAN;
ByteString buffer = null;
@Override
public Iterable<Either<ByteString, ByteString>> onCommand(
ByteString cmd) {
final int length = cmd.length() + 4;
if (length > maxSize) {
return new ArrayList<Either<ByteString, ByteString>>(0);
}
final ByteStringBuilder bb = new ByteStringBuilder();
bb.putInt(length, byteOrder);
bb.append(cmd);
return singleCommand(bb.result());
}
@Override
public Iterable<Either<ByteString, ByteString>> onEvent(
ByteString event) {
final ArrayList<Either<ByteString, ByteString>> res =
new ArrayList<Either<ByteString, ByteString>>();
ByteString current = buffer == null ? event : buffer.concat(event);
while (true) {
if (current.length() == 0) {
buffer = null;
return res;
} else if (current.length() < 4) {
buffer = current;
return res;
} else {
final int length = current.iterator().getInt(byteOrder);
if (length > maxSize)
throw new IllegalArgumentException(
"received too large frame of size " + length + " (max = "
+ maxSize + ")");
if (current.length() < length) {
buffer = current;
return res;
} else {
res.add(makeEvent(current.slice(4, length)));
current = current.drop(length);
}
}
}
}
});
}
}
//#frame

View file

@ -0,0 +1,45 @@
/**
* Copyright (C) 2013 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.io.japi;
//#message
public class Message {
static public class Person {
private final String first;
private final String last;
public Person(String first, String last) {
this.first = first;
this.last = last;
}
public String getFirst() {
return first;
}
public String getLast() {
return last;
}
}
private final Person[] persons;
private final double[] happinessCurve;
public Message(Person[] persons, double[] happinessCurve) {
this.persons = persons;
this.happinessCurve = happinessCurve;
}
public Person[] getPersons() {
return persons;
}
public double[] getHappinessCurve() {
return happinessCurve;
}
}
//#message

View file

@ -0,0 +1,118 @@
/**
* Copyright (C) 2013 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.io.japi;
import java.nio.ByteOrder;
import java.util.Collections;
import scala.concurrent.duration.Duration;
import scala.concurrent.duration.FiniteDuration;
import scala.util.Either;
import akka.actor.ActorRef;
import akka.io.AbstractSymmetricPipePair;
import akka.io.PipePairFactory;
import akka.io.SymmetricPipePair;
import akka.io.SymmetricPipelineStage;
import akka.util.ByteIterator;
import akka.util.ByteString;
import akka.util.ByteStringBuilder;
//#format
public class MessageStage extends
SymmetricPipelineStage<HasByteOrder, Message, ByteString> {
@Override
public SymmetricPipePair<Message, ByteString> apply(final HasByteOrder context) {
return PipePairFactory
.create(context, new AbstractSymmetricPipePair<Message, ByteString>() {
final ByteOrder byteOrder = context.byteOrder();
private void putString(ByteStringBuilder builder, String str) {
final byte[] bytes = ByteString.fromString(str, "UTF-8").toArray();
builder.putInt(bytes.length, byteOrder);
builder.putBytes(bytes);
}
@Override
public Iterable<Either<Message, ByteString>> onCommand(Message cmd) {
final ByteStringBuilder builder = new ByteStringBuilder();
builder.putInt(cmd.getPersons().length, byteOrder);
for (Message.Person p : cmd.getPersons()) {
putString(builder, p.getFirst());
putString(builder, p.getLast());
}
builder.putInt(cmd.getHappinessCurve().length, byteOrder);
builder.putDoubles(cmd.getHappinessCurve(), byteOrder);
return singleCommand(builder.result());
}
//#decoding-omitted
//#decoding
private String getString(ByteIterator iter) {
final int length = iter.getInt(byteOrder);
final byte[] bytes = new byte[length];
iter.getBytes(bytes);
return ByteString.fromArray(bytes).utf8String();
}
@Override
public Iterable<Either<Message, ByteString>> onEvent(ByteString evt) {
final ByteIterator iter = evt.iterator();
final int personLength = iter.getInt(byteOrder);
final Message.Person[] persons = new Message.Person[personLength];
for (int i = 0; i < personLength; ++i) {
persons[i] = new Message.Person(getString(iter), getString(iter));
}
final int curveLength = iter.getInt(byteOrder);
final double[] curve = new double[curveLength];
iter.getDoubles(curve, byteOrder);
// verify that this was all; could be left out to allow future
// extensions
assert iter.isEmpty();
return singleEvent(new Message(persons, curve));
}
//#decoding
ActorRef target = null;
//#mgmt-ticks
private FiniteDuration lastTick = Duration.Zero();
@Override
public Iterable<Either<Message, ByteString>> onManagementCommand(Object cmd) {
//#omitted
if (cmd instanceof PipelineTest.SetTarget) {
target = ((PipelineTest.SetTarget) cmd).getRef();
} else if (cmd instanceof TickGenerator.Tick && target != null) {
target.tell(cmd, null);
}
//#omitted
if (cmd instanceof TickGenerator.Tick) {
final FiniteDuration timestamp = ((TickGenerator.Tick) cmd)
.getTimestamp();
System.out.println("time since last tick: "
+ timestamp.minus(lastTick));
lastTick = timestamp;
}
return Collections.emptyList();
}
//#mgmt-ticks
//#decoding-omitted
});
}
}
//#format

View file

@ -0,0 +1,167 @@
/**
* Copyright (C) 2013 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.io.japi;
import java.nio.ByteOrder;
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.Actor;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.actor.PoisonPill;
import akka.actor.Props;
import akka.actor.UntypedActorFactory;
import akka.io.AbstractPipelineContext;
import akka.io.PipelineFactory;
import akka.io.PipelineInjector;
import akka.io.PipelineSink;
import akka.io.PipelineStage;
import akka.testkit.JavaTestKit;
import akka.testkit.TestProbe;
import akka.util.ByteString;
public class PipelineTest {
//#message
final Message msg = new Message(
new Message.Person[] {
new Message.Person("Alice", "Gibbons"),
new Message.Person("Bob", "Sparseley")
},
new double[] { 1.0, 3.0, 5.0 });
//#message
//#byteorder
class Context extends AbstractPipelineContext implements HasByteOrder {
@Override
public ByteOrder byteOrder() {
return java.nio.ByteOrder.BIG_ENDIAN;
}
}
final Context ctx = new Context();
//#byteorder
static ActorSystem system = null;
@BeforeClass
public static void setup() {
system = ActorSystem.create("PipelineTest");
}
@AfterClass
public static void teardown() {
system.shutdown();
}
@Test
public void demonstratePipeline() throws Exception {
final TestProbe probe = TestProbe.apply(system);
final ActorRef commandHandler = probe.ref();
final ActorRef eventHandler = probe.ref();
//#build-sink
final PipelineStage<Context, Message, ByteString, Message, ByteString> stages =
PipelineStage.sequence(
new MessageStage(),
new LengthFieldFrame(10000)
);
final PipelineSink<ByteString, Message> sink =
new PipelineSink<ByteString, Message>() {
@Override
public void onCommand(ByteString cmd) throws Throwable {
commandHandler.tell(cmd, null);
}
@Override
public void onEvent(Message evt) throws Throwable {
eventHandler.tell(evt, null);
}
};
final PipelineInjector<Message, ByteString> injector =
PipelineFactory.buildWithSink(ctx, stages, sink);
injector.injectCommand(msg);
//#build-sink
final ByteString encoded = probe.expectMsgClass(ByteString.class);
injector.injectEvent(encoded);
final Message decoded = probe.expectMsgClass(Message.class);
assert msg == decoded;
}
static class SetTarget {
final ActorRef ref;
public SetTarget(ActorRef ref) {
super();
this.ref = ref;
}
public ActorRef getRef() {
return ref;
}
}
@Test
public void testTick() {
new JavaTestKit(system) {
{
final ActorRef proc = system.actorOf(new Props(
new UntypedActorFactory() {
private static final long serialVersionUID = 1L;
@Override
public Actor create() throws Exception {
return new Processor(getRef(), getRef()) {
@Override
public void onReceive(Object obj) throws Exception {
if (obj.equals("fail!")) {
throw new RuntimeException("FAIL!");
}
super.onReceive(obj);
}
};
}
}), "processor");
expectMsgClass(TickGenerator.Tick.class);
proc.tell(msg, null);
final ByteString encoded = expectMsgClass(ByteString.class);
proc.tell(encoded, null);
final Message decoded = expectMsgClass(Message.class);
assert msg == decoded;
new Within(Duration.create(1500, TimeUnit.MILLISECONDS),
Duration.create(3, TimeUnit.SECONDS)) {
protected void run() {
expectMsgClass(TickGenerator.Tick.class);
expectMsgClass(TickGenerator.Tick.class);
}
};
proc.tell("fail!", null);
new Within(Duration.create(1700, TimeUnit.MILLISECONDS),
Duration.create(3, TimeUnit.SECONDS)) {
protected void run() {
expectMsgClass(TickGenerator.Tick.class);
expectMsgClass(TickGenerator.Tick.class);
proc.tell(PoisonPill.getInstance(), null);
expectNoMsg();
}
};
}
};
}
}

View file

@ -0,0 +1,93 @@
/**
* Copyright (C) 2013 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.io.japi;
import java.nio.ByteOrder;
import java.util.concurrent.TimeUnit;
import akka.actor.ActorContext;
import akka.actor.ActorRef;
import akka.actor.UntypedActor;
import akka.io.AbstractPipelineContext;
import akka.io.PipelineFactory;
import akka.io.PipelineInjector;
import akka.io.PipelineSink;
import akka.io.PipelineStage;
import akka.util.ByteString;
import scala.concurrent.duration.*;
//#actor
public class Processor extends UntypedActor {
private class Context extends AbstractPipelineContext
implements HasByteOrder, HasActorContext {
@Override
public ActorContext getContext() {
return Processor.this.getContext();
}
@Override
public ByteOrder byteOrder() {
return java.nio.ByteOrder.BIG_ENDIAN;
}
}
final Context ctx = new Context();
final FiniteDuration interval = Duration.apply(1, TimeUnit.SECONDS);
final PipelineStage<Context, Message, ByteString, Message, ByteString> stages =
PipelineStage.sequence(
// Java 7 can infer these types, Java 6 cannot
PipelineStage.<Context, Message, Message, ByteString, Message, Message, ByteString> sequence( //
new TickGenerator<Message, Message>(interval), //
new MessageStage()), //
new LengthFieldFrame(10000));
private final ActorRef evts;
private final ActorRef cmds;
final PipelineInjector<Message, ByteString> injector = PipelineFactory
.buildWithSink(ctx, stages, new PipelineSink<ByteString, Message>() {
@Override
public void onCommand(ByteString cmd) {
cmds.tell(cmd, getSelf());
}
@Override
public void onEvent(Message evt) {
evts.tell(evt, getSelf());
}
});
public Processor(ActorRef cmds, ActorRef evts) throws Exception {
this.cmds = cmds;
this.evts = evts;
}
//#omitted
@Override
public void preStart() throws Exception {
injector.managementCommand(new PipelineTest.SetTarget(cmds));
}
//#omitted
@Override
public void onReceive(Object obj) throws Exception {
if (obj instanceof Message) {
injector.injectCommand((Message) obj);
} else if (obj instanceof ByteString) {
injector.injectEvent((ByteString) obj);
} else if (obj instanceof TickGenerator.Trigger) {
injector.managementCommand(obj);
}
}
}
//#actor

View file

@ -0,0 +1,86 @@
/**
* Copyright (C) 2013 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.io.japi;
import java.util.Collections;
import scala.concurrent.duration.Deadline;
import scala.concurrent.duration.FiniteDuration;
import scala.util.Either;
import akka.actor.ActorSystem;
import akka.io.AbstractPipePair;
import akka.io.PipePair;
import akka.io.PipePairFactory;
import akka.io.PipelineStage;
//#tick-generator
public class TickGenerator<Cmd, Evt> extends
PipelineStage<HasActorContext, Cmd, Cmd, Evt, Evt> {
public static interface Trigger {};
public static class Tick implements Trigger {
final FiniteDuration timestamp;
public Tick(FiniteDuration timestamp) {
super();
this.timestamp = timestamp;
}
public FiniteDuration getTimestamp() {
return timestamp;
}
}
private final FiniteDuration interval;
public TickGenerator(FiniteDuration interval) {
this.interval = interval;
}
@Override
public PipePair<Cmd, Cmd, Evt, Evt> apply(final HasActorContext ctx) {
return PipePairFactory.create(ctx,
new AbstractPipePair<Cmd, Cmd, Evt, Evt>() {
private final Trigger trigger = new Trigger() {
public String toString() {
return "Tick[" + ctx.getContext().self().path() + "]";
}
};
private void schedule() {
final ActorSystem system = ctx.getContext().system();
system.scheduler().scheduleOnce(interval,
ctx.getContext().self(), trigger, system.dispatcher(), null);
}
{
schedule();
}
@Override
public Iterable<Either<Evt, Cmd>> onCommand(Cmd cmd) {
return singleCommand(cmd);
}
@Override
public Iterable<Either<Evt, Cmd>> onEvent(Evt evt) {
return singleEvent(evt);
}
@Override
public Iterable<Either<Evt, Cmd>> onManagementCommand(Object cmd) {
if (cmd == trigger) {
ctx.getContext().self().tell(new Tick(Deadline.now().time()), null);
schedule();
}
return Collections.emptyList();
}
});
}
}
//#tick-generator

View file

@ -99,6 +99,250 @@ Compatibility with java.io
A ``ByteStringBuilder`` can be wrapped in a ``java.io.OutputStream`` via the ``asOutputStream`` method. Likewise, ``ByteIterator`` can we wrapped in a ``java.io.InputStream`` via ``asInputStream``. Using these, ``akka.io`` applications can integrate legacy code based on ``java.io`` streams.
Encoding and decoding binary data
---------------------------------
Akka adopted and adapted the implementation of data processing pipelines found
in the ``spray-io`` module. The idea is that encoding and decoding often
go hand in hand and keeping the code pertaining to one protocol layer together
is deemed more important than writing down the complete read side—say—in the
iteratee style in one go; pipelines encourage packaging the stages in a form
which lends itself better to reuse in a protocol stack. Another reason for
choosing this abstraction is that it is at times necessary to change the
behavior of encoding and decoding within a stage based on a message streams
state, and pipeline stages allow communication between the read and write
halves quite naturally.
The actual byte-fiddling can be done within pipeline stages, for example using
the rich API of :class:`ByteIterator` and :class:`ByteStringBuilder` as shown
below. All these activities are synchronous transformations which benefit
greatly from CPU affinity to make good use of those data caches. Therefore the
design of the pipeline infrastructure is completely synchronous, every stages
handler code can only directly return the events and/or commands resulting from
an input, there are no callbacks. Exceptions thrown within a pipeline stage
will abort processing of the whole pipeline under the assumption that
recoverable error conditions will be signaled in-band to the next stage instead
of raising an exception.
An overall “logical” pipeline can span multiple execution contexts, for example
starting with the low-level protocol layers directly within an actor handling
the reads and writes to a TCP connection and then being passed to a number of
higher-level actors which do the costly application level processing. This is
supported by feeding the generated events into a sink which sends them to
another actor, and that other actor will then upon reception feed them into its
own pipeline.
Introducing the Sample Protocol
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
In the following the process of implementing a protocol stack using pipelines
is demonstrated on the following simple example:
.. code-block:: text
frameLen: Int
persons: Int
persons times {
first: String
last: String
}
points: Int
points times Double
mapping to the following data type:
.. includecode:: code/docs/io/japi/Message.java#message
We will split the handling of this protocol into two parts: the frame-length
encoding handles the buffering necessary on the read side and the actual
encoding of the frame contents is done in a separate stage.
Building a Pipeline Stage
^^^^^^^^^^^^^^^^^^^^^^^^^
As a common example, which is also included in the ``akka-actor`` package, let
us look at a framing protocol which works by prepending a length field to each
message (the following is a simplified version for demonstration purposes, the
real implementation is more configurable and implemented in Scala).
.. includecode:: code/docs/io/japi/LengthFieldFrame.java
:include: frame
In the end a pipeline stage is nothing more than a set of three methods: one
transforming commands arriving from above, one transforming events arriving
from below and the third transforming incoming management commands (not shown
here, see below for more information). The result of the transformation can in
either case be a sequence of commands flowing downwards or events flowing
upwards (or a combination thereof).
In the case above the data type for commands and events are equal as both
functions operate only on ``ByteString``, and the transformation does not
change that type because it only adds or removes four octets at the front.
The pair of command and event transformation functions is represented by an
object of type :class:`AbstractPipePair`, or in this case a
:class:`AbstractSymmetricPipePair`. This object could benefit from knowledge
about the context it is running in, for example an :class:`Actor`, and this
context is introduced by making a :class:`PipelineStage` be a factory for
producing a :class:`PipePair`. The factory method is called :meth:`apply` (a
Scala tradition) and receives the context object as its argument. The
implementation of this factory method could now make use of the context in
whatever way it sees fit, you will see an example further down.
Manipulating ByteStrings
^^^^^^^^^^^^^^^^^^^^^^^^
The second stage of our sample protocol stack illustrates in more depth what
showed only a little in the pipeline stage built above: constructing and
deconstructing byte strings. Let us first take a look at the encoder:
.. includecode:: code/docs/io/japi/MessageStage.java
:include: format
:exclude: decoding-omitted,omitted
Note how the byte order to be used by this stage is fixed in exactly one place,
making it impossible get wrong between commands and events; the way how the
byte order is passed into the stage demonstrates one possible use for the
stages ``context`` parameter.
The basic tool for constucting a :class:`ByteString` is a
:class:`ByteStringBuilder`. This builder is specialized for concatenating byte
representations of the primitive data types like ``Int`` and ``Double`` or
arrays thereof. Encoding a ``String`` requires a bit more work because not
only the sequence of bytes needs to be encoded but also the length, otherwise
the decoding stage would not know where the ``String`` terminates. When all
values making up the :class:`Message` have been appended to the builder, we
simply pass the resulting :class:`ByteString` on to the next stage as a command
using the optimized :meth:`singleCommand` facility.
.. warning::
The :meth:`singleCommand` and :meth:`singleEvent` methods provide a way to
generate responses which transfer exactly one result from one pipeline stage
to the next without suffering the overhead of object allocations. This means
that the returned collection object will not work for anything else (you will
get :class:`ClassCastExceptions`!) and this facility can only be used *EXACTLY
ONCE* during the processing of one input (command or event).
Now let us look at the decoder side:
.. includecode:: code/docs/io/japi/MessageStage.java
:include: decoding
The decoding side does the same things that the encoder does in the same order,
it just uses a :class:`ByteIterator` to retrieve primitive data types or arrays
of those from the underlying :class:`ByteString`. And in the end it hands the
assembled :class:`Message` as an event to the next stage using the optimized
:meth:`singleEvent` facility (see warning above).
Building a Pipeline
^^^^^^^^^^^^^^^^^^^
Given the two pipeline stages introduced in the sections above we can now put
them to some use. First we define some message to be encoded:
.. includecode:: code/docs/io/japi/PipelineTest.java
:include: message
Then we need to create a pipeline context which satisfies our declared needs:
.. includecode:: code/docs/io/japi/PipelineTest.java
:include: byteorder
Building the pipeline and encoding this message then is quite simple:
.. includecode:: code/docs/io/japi/PipelineTest.java
:include: build-sink
First we *sequence* the two stages, i.e. attach them such that the output of
one becomes the input of the other. Then we create a :class:`PipelineSink`
which is essentially a callback interface for what shall happen with the
encoded commands or decoded events, respectively. Then we build the pipeline
using the :class:`PipelineFactory`, which returns an interface for feeding
commands and events into this pipeline instance. As a demonstration of how to
use this, we simply encode the message shown above and the resulting
:class:`ByteString` will then be sent to the ``commandHandler`` actor. Decoding
works in the same way, only using :meth:`injectEvent`.
Injecting into a pipeline using a :class:`PipelineInjector` will catch
exceptions resulting from processing the input, in which case the exception
(there can only be one per injection) is passed into the respective sink. The
default implementation of :meth:`onCommandFailure` and :meth:`onEventFailure`
will re-throw the exception (whence originates the ``throws`` declaration of
the ``inject*`` method).
Using the Pipelines Context
^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Up to this point there was always a parameter ``ctx`` which was used when
constructing a pipeline, but it was not explained in full. The context is a
piece of information which is made available to all stages of a pipeline. The
context may also carry behavior, provide infrastructure or helper methods etc.
It should be noted that the context is bound to the pipeline and as such must
not be accessed concurrently from different threads unless care is taken to
properly synchronize such access. Since the context will in many cases be
provided by an actor it is not recommended to share this context with code
executing outside of the actors message handling.
.. warning::
A PipelineContext instance *MUST NOT* be used by two different pipelines
since it contains mutable fields which are used during message processing.
Using Management Commands
^^^^^^^^^^^^^^^^^^^^^^^^^
Since pipeline stages do not have any reference to the pipeline or even to
their neighbors they cannot directly effect the injection of commands or events
outside of their normal processing. But sometimes things need to happen driven
by a timer, for example. In this case the timer would need to cause sending
tick messages to the whole pipeline, and those stages which wanted to receive
them would act upon those. In order to keep the type signatures for events and
commands useful, such external triggers are sent out-of-band, via a different
channel—the management port. One example which makes use of this facility is
the :class:`TickGenerator` which comes included with ``akka-actor`` (this is a
transcription of the Scala version which is actually included in the
``akka-actor`` JAR):
.. includecode:: code/docs/io/japi/HasActorContext.java#actor-context
.. includecode:: code/docs/io/japi/TickGenerator.java#tick-generator
This pipeline stage is to be used within an actor, and it will make use of this
context in order to schedule the delivery of ``Tick`` messages; the actor is
then supposed to feed these messages into the management port of the pipeline.
An example could look like this:
.. includecode:: code/docs/io/japi/Processor.java
:include: actor
:exclude: omitted
This actor extends our well-known pipeline with the tick generator and attaches
the outputs to functions which send commands and events to actors for further
processing. The pipeline stages will then all receive on ``Tick`` per second
which can be used like so:
.. includecode:: code/docs/io/japi/MessageStage.java
:include: mgmt-ticks
:exclude: omitted
.. note::
Management commands are delivered to all stages of a pipeline “effectively
parallel”, like on a broadcast medium. No code will actually run concurrently
since a pipeline is strictly single-threaded, but the order in which these
commands are processed is not specified.
The intended purpose of management commands is for each stage to define its
special command types and then listen only to those (where the aforementioned
``Tick`` message is a useful counter-example), exactly like sending packets on
a wifi network where every station receives all traffic but reacts only to
those messages which are destined for it.
If you need all stages to react upon something in their defined order, then
this must be modeled either as a command or event, i.e. it will be part of the
“business” type of the pipeline.
Using TCP
---------