Merge branch 'master' into wip-2053d-actorbased-remote-drewhk

Conflicts:
	akka-docs/rst/java/code/docs/serialization/SerializationDocTestBase.java
	akka-docs/rst/scala/code/docs/serialization/SerializationDocSpec.scala
	akka-remote-tests/src/main/scala/akka/remote/testconductor/NetworkFailureInjector.scala
	akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala
This commit is contained in:
Endre Sándor Varga 2012-12-18 15:15:01 +01:00
commit 55be17419e
213 changed files with 5004 additions and 1940 deletions

View file

@ -31,20 +31,18 @@ A Simple Cluster Example
^^^^^^^^^^^^^^^^^^^^^^^^
The following small program together with its configuration starts an ``ActorSystem``
with the Cluster extension enabled. It joins the cluster and logs some membership events.
with the Cluster enabled. It joins the cluster and logs some membership events.
Try it out:
1. Add the following ``application.conf`` in your project, place it in ``src/main/resources``:
.. literalinclude:: ../../../akka-samples/akka-sample-cluster/src/main/resources/application.conf
:language: none
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/resources/application.conf#cluster
To enable cluster capabilities in your Akka project you should, at a minimum, add the :ref:`remoting-java`
settings, but with ``akka.cluster.ClusterActorRefProvider``.
The ``akka.cluster.seed-nodes`` and cluster extension should normally also be added to your
``application.conf`` file.
The ``akka.cluster.seed-nodes`` should normally also be added to your ``application.conf`` file.
The seed nodes are configured contact points for initial, automatic, join of the cluster.
@ -241,6 +239,25 @@ frontend nodes and 3 backend nodes::
.. note:: The above example should probably be designed as two separate, frontend/backend, clusters, when there is a `cluster client for decoupling clusters <https://www.assembla.com/spaces/akka/tickets/1165>`_.
How To Startup when Cluster Size Reached
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
A common use case is to start actors after the cluster has been initialized,
members have joined, and the cluster has reached a certain size.
With a configuration option you can define required number of members
before the leader changes member status of 'Joining' members to 'Up'.
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/resources/factorial.conf#min-nr-of-members
You can start the actors in a ``registerOnMemberUp`` callback, which will
be invoked when the current member status is changed tp 'Up', i.e. the cluster
has at least the defined number of members.
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialFrontendMain.java#registerOnUp
This callback can be used for other things than starting actors.
Failure Detector
^^^^^^^^^^^^^^^^
@ -438,6 +455,107 @@ service nodes and 1 client::
.. note:: The above example, especially the last part, will be simplified when the cluster handles automatic actor partitioning.
Cluster Metrics
^^^^^^^^^^^^^^^
The member nodes of the cluster collects system health metrics and publishes that to other nodes and to
registered subscribers. This information is primarily used for load-balancing routers.
Hyperic Sigar
-------------
The built-in metrics is gathered from JMX MBeans, and optionally you can use `Hyperic Sigar <http://www.hyperic.com/products/sigar>`_
for a wider and more accurate range of metrics compared to what can be retrieved from ordinary MBeans.
Sigar is using a native OS library. To enable usage of Sigar you need to add the directory of the native library to
``-Djava.libarary.path=<path_of_sigar_libs>`` add the following dependency::
<dependency>
<groupId>org.hyperic</groupId>
<artifactId>sigar</artifactId>
<version>@sigarVersion@</version>
</dependency>
Adaptive Load Balancing
-----------------------
The ``AdaptiveLoadBalancingRouter`` performs load balancing of messages to cluster nodes based on the cluster metrics data.
It uses random selection of routees with probabilities derived from the remaining capacity of the corresponding node.
It can be configured to use a specific MetricsSelector to produce the probabilities, a.k.a. weights:
* ``heap`` / ``HeapMetricsSelector`` - Used and max JVM heap memory. Weights based on remaining heap capacity; (max - used) / max
* ``load`` / ``SystemLoadAverageMetricsSelector`` - System load average for the past 1 minute, corresponding value can be found in ``top`` of Linux systems. The system is possibly nearing a bottleneck if the system load average is nearing number of cpus/cores. Weights based on remaining load capacity; 1 - (load / processors)
* ``cpu`` / ``CpuMetricsSelector`` - CPU utilization in percentage, sum of User + Sys + Nice + Wait. Weights based on remaining cpu capacity; 1 - utilization
* ``mix`` / ``MixMetricsSelector`` - Combines heap, cpu and load. Weights based on mean of remaining capacity of the combined selectors.
* Any custom implementation of ``akka.cluster.routing.MetricsSelector``
The collected metrics values are smoothed with `exponential weighted moving average <http://en.wikipedia.org/wiki/Moving_average#Exponential_moving_average>`_. In the :ref:`cluster_configuration_java` you can adjust how quickly past data is decayed compared to new data.
Let's take a look at this router in action.
In this example the following imports are used:
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialBackend.java#imports
The backend worker that performs the factorial calculation:
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialBackend.java#backend
The frontend that receives user jobs and delegates to the backends via the router:
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialFrontend.java#frontend
As you can see, the router is defined in the same way as other routers, and in this case it's configured as follows:
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/resources/application.conf#adaptive-router
It's only router type ``adaptive`` and the ``metrics-selector`` that is specific to this router, other things work
in the same way as other routers.
The same type of router could also have been defined in code:
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialFrontend.java#router-lookup-in-code
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/FactorialFrontend.java#router-deploy-in-code
This example is included in ``akka-samples/akka-sample-cluster`` and you can try it by copying the
`source <@github@/akka-samples/akka-sample-cluster>`_ to your
maven project, defined as in :ref:`cluster_simple_example_java`.
Run it by starting nodes in different terminal windows. For example, starting 3 backend nodes and
one frontend::
mvn exec:java \
-Dexec.mainClass="sample.cluster.factorial.FactorialBackendMain" \
-Dexec.args="2551"
mvn exec:java \
-Dexec.mainClass="sample.cluster.factorial.FactorialBackendMain" \
-Dexec.args="2552"
mvn exec:java \
-Dexec.mainClass="sample.cluster.factorial.FactorialBackendMain"
mvn exec:java \
-Dexec.mainClass="sample.cluster.factorial.FactorialFrontendMain"
Press ctrl-c in the terminal window of the frontend to stop the factorial calculations.
Subscribe to Metrics Events
---------------------------
It's possible to subscribe to the metrics events directly to implement other functionality.
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/java/sample/cluster/factorial/japi/MetricsListener.java#metrics-listener
Custom Metrics Collector
------------------------
You can plug-in your own metrics collector instead of
``akka.cluster.SigarMetricsCollector`` or ``akka.cluster.JmxMetricsCollector``. Look at those two implementations
for inspiration. The implementation class can be defined in the :ref:`cluster_configuration_java`.
.. _cluster_jmx_java:
@ -474,15 +592,16 @@ Run it without parameters to see instructions about how to use the script::
leave <node-url> - Sends a request for node with URL to LEAVE the cluster
down <node-url> - Sends a request for marking node with URL as DOWN
member-status - Asks the member node for its current status
members - Asks the cluster for addresses of current members
unreachable - Asks the cluster for addresses of unreachable members
cluster-status - Asks the cluster for its current status (member ring,
unavailable nodes, meta data etc.)
leader - Asks the cluster who the current leader is
is-singleton - Checks if the cluster is a singleton cluster (single
node cluster)
is-available - Checks if the member node is available
is-running - Checks if the member node is running
has-convergence - Checks if there is a cluster convergence
Where the <node-url> should be on the format of 'akka://actor-system-name@hostname:port'
Where the <node-url> should be on the format of
'akka://actor-system-name@hostname:port'
Examples: bin/akka-cluster localhost:9999 is-available
bin/akka-cluster localhost:9999 join akka://MySystem@darkstar:2552
@ -522,7 +641,7 @@ introduce the extra overhead of another thread.
::
# shorter tick-duration of default scheduler when using cluster
akka.scheduler.tick-duration.tick-duration = 33ms
akka.scheduler.tick-duration = 33ms

View file

@ -25,20 +25,18 @@ A Simple Cluster Example
^^^^^^^^^^^^^^^^^^^^^^^^
The following small program together with its configuration starts an ``ActorSystem``
with the Cluster extension enabled. It joins the cluster and logs some membership events.
with the Cluster enabled. It joins the cluster and logs some membership events.
Try it out:
1. Add the following ``application.conf`` in your project, place it in ``src/main/resources``:
.. literalinclude:: ../../../akka-samples/akka-sample-cluster/src/main/resources/application.conf
:language: none
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/resources/application.conf#cluster
To enable cluster capabilities in your Akka project you should, at a minimum, add the :ref:`remoting-scala`
settings, but with ``akka.cluster.ClusterActorRefProvider``.
The ``akka.cluster.seed-nodes`` and cluster extension should normally also be added to your
``application.conf`` file.
The ``akka.cluster.seed-nodes`` should normally also be added to your ``application.conf`` file.
The seed nodes are configured contact points for initial, automatic, join of the cluster.
@ -214,6 +212,25 @@ frontend nodes and 3 backend nodes::
.. note:: The above example should probably be designed as two separate, frontend/backend, clusters, when there is a `cluster client for decoupling clusters <https://www.assembla.com/spaces/akka/tickets/1165>`_.
How To Startup when Cluster Size Reached
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
A common use case is to start actors after the cluster has been initialized,
members have joined, and the cluster has reached a certain size.
With a configuration option you can define required number of members
before the leader changes member status of 'Joining' members to 'Up'.
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/resources/factorial.conf#min-nr-of-members
You can start the actors in a ``registerOnMemberUp`` callback, which will
be invoked when the current member status is changed tp 'Up', i.e. the cluster
has at least the defined number of members.
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/factorial/FactorialSample.scala#registerOnUp
This callback can be used for other things than starting actors.
Failure Detector
^^^^^^^^^^^^^^^^
@ -265,6 +282,8 @@ This is how the curve looks like for ``acceptable-heartbeat-pause`` configured t
.. image:: images/phi3.png
.. _cluster_aware_routers_scala:
Cluster Aware Routers
^^^^^^^^^^^^^^^^^^^^^
@ -397,6 +416,97 @@ service nodes and 1 client::
.. note:: The above example, especially the last part, will be simplified when the cluster handles automatic actor partitioning.
Cluster Metrics
^^^^^^^^^^^^^^^
The member nodes of the cluster collects system health metrics and publishes that to other nodes and to
registered subscribers. This information is primarily used for load-balancing routers.
Hyperic Sigar
-------------
The built-in metrics is gathered from JMX MBeans, and optionally you can use `Hyperic Sigar <http://www.hyperic.com/products/sigar>`_
for a wider and more accurate range of metrics compared to what can be retrieved from ordinary MBeans.
Sigar is using a native OS library. To enable usage of Sigar you need to add the directory of the native library to
``-Djava.libarary.path=<path_of_sigar_libs>`` add the following dependency::
"org.hyperic" % "sigar" % "@sigarVersion@"
Adaptive Load Balancing
-----------------------
The ``AdaptiveLoadBalancingRouter`` performs load balancing of messages to cluster nodes based on the cluster metrics data.
It uses random selection of routees with probabilities derived from the remaining capacity of the corresponding node.
It can be configured to use a specific MetricsSelector to produce the probabilities, a.k.a. weights:
* ``heap`` / ``HeapMetricsSelector`` - Used and max JVM heap memory. Weights based on remaining heap capacity; (max - used) / max
* ``load`` / ``SystemLoadAverageMetricsSelector`` - System load average for the past 1 minute, corresponding value can be found in ``top`` of Linux systems. The system is possibly nearing a bottleneck if the system load average is nearing number of cpus/cores. Weights based on remaining load capacity; 1 - (load / processors)
* ``cpu`` / ``CpuMetricsSelector`` - CPU utilization in percentage, sum of User + Sys + Nice + Wait. Weights based on remaining cpu capacity; 1 - utilization
* ``mix`` / ``MixMetricsSelector`` - Combines heap, cpu and load. Weights based on mean of remaining capacity of the combined selectors.
* Any custom implementation of ``akka.cluster.routing.MetricsSelector``
The collected metrics values are smoothed with `exponential weighted moving average <http://en.wikipedia.org/wiki/Moving_average#Exponential_moving_average>`_. In the :ref:`cluster_configuration_scala` you can adjust how quickly past data is decayed compared to new data.
Let's take a look at this router in action.
In this example the following imports are used:
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/factorial/FactorialSample.scala#imports
The backend worker that performs the factorial calculation:
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/factorial/FactorialSample.scala#backend
The frontend that receives user jobs and delegates to the backends via the router:
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/factorial/FactorialSample.scala#frontend
As you can see, the router is defined in the same way as other routers, and in this case it's configured as follows:
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/resources/application.conf#adaptive-router
It's only router type ``adaptive`` and the ``metrics-selector`` that is specific to this router, other things work
in the same way as other routers.
The same type of router could also have been defined in code:
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/factorial/FactorialSample.scala#router-lookup-in-code
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/factorial/FactorialSample.scala#router-deploy-in-code
This example is included in ``akka-samples/akka-sample-cluster``
and you can try by starting nodes in different terminal windows. For example, starting 3 backend nodes and one frontend::
sbt
project akka-sample-cluster-experimental
run-main sample.cluster.factorial.FactorialBackend 2551
run-main sample.cluster.factorial.FactorialBackend 2552
run-main sample.cluster.factorial.FactorialBackend
run-main sample.cluster.factorial.FactorialFrontend
Press ctrl-c in the terminal window of the frontend to stop the factorial calculations.
Subscribe to Metrics Events
---------------------------
It's possible to subscribe to the metrics events directly to implement other functionality.
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/scala/sample/cluster/factorial/FactorialSample.scala#metrics-listener
Custom Metrics Collector
------------------------
You can plug-in your own metrics collector instead of
``akka.cluster.SigarMetricsCollector`` or ``akka.cluster.JmxMetricsCollector``. Look at those two implementations
for inspiration. The implementation class can be defined in the :ref:`cluster_configuration_scala`.
How to Test
^^^^^^^^^^^
@ -488,15 +598,16 @@ Run it without parameters to see instructions about how to use the script::
leave <node-url> - Sends a request for node with URL to LEAVE the cluster
down <node-url> - Sends a request for marking node with URL as DOWN
member-status - Asks the member node for its current status
members - Asks the cluster for addresses of current members
unreachable - Asks the cluster for addresses of unreachable members
cluster-status - Asks the cluster for its current status (member ring,
unavailable nodes, meta data etc.)
leader - Asks the cluster who the current leader is
is-singleton - Checks if the cluster is a singleton cluster (single
node cluster)
is-available - Checks if the member node is available
is-running - Checks if the member node is running
has-convergence - Checks if there is a cluster convergence
Where the <node-url> should be on the format of 'akka://actor-system-name@hostname:port'
Where the <node-url> should be on the format of
'akka://actor-system-name@hostname:port'
Examples: bin/akka-cluster localhost:9999 is-available
bin/akka-cluster localhost:9999 join akka://MySystem@darkstar:2552
@ -536,7 +647,7 @@ introduce the extra overhead of another thread.
::
# shorter tick-duration of default scheduler when using cluster
akka.scheduler.tick-duration.tick-duration = 33ms
akka.scheduler.tick-duration = 33ms

View file

@ -84,9 +84,9 @@ Gossip
The cluster membership used in Akka is based on Amazon's `Dynamo`_ system and
particularly the approach taken in Basho's' `Riak`_ distributed database.
Cluster membership is communicated using a `Gossip Protocol`_, where the current
state of the cluster is gossiped randomly through the cluster. Joining a cluster
is initiated by issuing a ``Join`` command to one of the nodes in the cluster to
join.
state of the cluster is gossiped randomly through the cluster, with preference to
members that have not seen the latest version. Joining a cluster is initiated
by issuing a ``Join`` command to one of the nodes in the cluster to join.
.. _Gossip Protocol: http://en.wikipedia.org/wiki/Gossip_protocol
.. _Dynamo: http://www.allthingsdistributed.com/files/amazon-dynamo-sosp2007.pdf
@ -209,8 +209,7 @@ node to initiate a round of gossip with. The choice of node is random but can
also include extra gossiping nodes with either newer or older state versions.
The gossip overview contains the current state version for all nodes and also a
list of unreachable nodes. Whenever a node receives a gossip overview it updates
the `Failure Detector`_ with the liveness information.
list of unreachable nodes.
The nodes defined as ``seed`` nodes are just regular member nodes whose only
"special role" is to function as contact points in the cluster.

View file

@ -127,7 +127,7 @@ Add texlive bin to $PATH:
::
/usr/local/texlive/2010basic/bin/universal-darwin
/usr/local/texlive/2012basic/bin/universal-darwin
Add missing tex packages:
@ -140,10 +140,3 @@ Add missing tex packages:
sudo tlmgr install wrapfig
sudo tlmgr install helvetic
sudo tlmgr install courier
Link the akka pygments style:
::
cd /usr/local/Cellar/python/2.7.1/lib/python2.7/site-packages/pygments/styles
ln -s /path/to/akka/akka-docs/themes/akka/pygments/akka.py akka.py

View file

@ -13,9 +13,9 @@ Prior to Java 5, the Java Memory Model (JMM) was ill defined. It was possible to
shared memory was accessed by multiple threads, such as:
* a thread not seeing values written by other threads: a visibility problem
* a thread observing 'impossible' behavior of other threads, caused by instructions not being executed in the order
expected: an instruction reordering problem.
* a thread observing 'impossible' behavior of other threads, caused by
instructions not being executed in the order expected: an instruction
reordering problem.
With the implementation of JSR 133 in Java 5, a lot of these issues have been resolved. The JMM is a set of rules based
on the "happens-before" relation, which constrain when one memory access must happen before another, and conversely,
@ -120,4 +120,4 @@ Since Akka runs on the JVM there are still some rules to be followed.
}
}
* Messages **should** be immutable, this is to avoid the shared mutable state trap.
* Messages **should** be immutable, this is to avoid the shared mutable state trap.

View file

@ -24,7 +24,7 @@ and then there's the whole package, the Akka Microkernel, which is a standalone
container to deploy your Akka application in. With CPUs growing more and more
cores every cycle, Akka is the alternative that provides outstanding performance
even if you're only running it on one machine. Akka also supplies a wide array
of concurrency-paradigms, allowing for users to choose the right tool for the
of concurrency-paradigms, allowing users to choose the right tool for the
job.

View file

@ -388,6 +388,8 @@ URI options
The following URI options are supported:
.. tabularcolumns:: |l|l|l|L|
+--------------+----------+---------+------------------------------------------------+
| Name | Type | Default | Description |
+==============+==========+=========+================================================+

View file

@ -79,7 +79,7 @@ public class SchedulerDocTestBase {
//to the tickActor after 0ms repeating every 50ms
Cancellable cancellable = system.scheduler().schedule(Duration.Zero(),
Duration.create(50, TimeUnit.MILLISECONDS), tickActor, "Tick",
system.dispatcher());
system.dispatcher(), null);
//This cancels further Ticks to be sent
cancellable.cancel();

View file

@ -148,7 +148,7 @@ public class FaultHandlingDocSample {
progressListener = getSender();
getContext().system().scheduler().schedule(
Duration.Zero(), Duration.create(1, "second"), getSelf(), Do,
getContext().dispatcher()
getContext().dispatcher(), null
);
} else if (msg.equals(Do)) {
counterService.tell(new Increment(1), getSelf());

View file

@ -43,10 +43,10 @@ import scala.concurrent.ExecutionContext$;
//#imports8
import static akka.pattern.Patterns.after;
import java.util.Arrays;
//#imports8
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
@ -564,7 +564,7 @@ public class FutureDocTestBase {
return "foo";
}
}, ec);
Future<String> result = future.either(delayed);
Future<String> result = Futures.firstCompletedOf(Arrays.asList(future, delayed), ec);
//#after
Await.result(result, Duration.create(2, SECONDS));
}

View file

@ -70,7 +70,7 @@ public class RouterViaProgramExample {
int upperBound = 15;
DefaultResizer resizer = new DefaultResizer(lowerBound, upperBound);
ActorRef router3 = system.actorOf(
new Props(ExampleActor.class).withRouter(new RoundRobinRouter(nrOfInstances)));
new Props(ExampleActor.class).withRouter(new RoundRobinRouter(resizer)));
//#programmaticRoutingWithResizer
for (int i = 1; i <= 6; i++) {
router3.tell(new ExampleActor.Message(i), null);

View file

@ -35,7 +35,7 @@ public class SchedulerPatternTest {
private final Cancellable tick = getContext().system().scheduler().schedule(
Duration.create(500, TimeUnit.MILLISECONDS),
Duration.create(1000, TimeUnit.MILLISECONDS),
getSelf(), "tick", getContext().dispatcher());
getSelf(), "tick", getContext().dispatcher(), null);
//#schedule-constructor
// this variable and constructor is declared here to not show up in the docs
final ActorRef target;

View file

@ -0,0 +1,108 @@
package docs.pattern;
import java.util.concurrent.TimeoutException;
import scala.concurrent.Future;
import scala.concurrent.duration.Duration;
import akka.actor.ActorKilledException;
import akka.actor.ActorRef;
import akka.actor.ActorRefFactory;
import akka.actor.Cancellable;
import akka.actor.OneForOneStrategy;
import akka.actor.Props;
import akka.actor.Scheduler;
import akka.actor.Status;
import akka.actor.SupervisorStrategy;
import akka.actor.SupervisorStrategy.Directive;
import akka.actor.Terminated;
import akka.actor.UntypedActor;
import akka.japi.Function;
import akka.pattern.Patterns;
import akka.util.Timeout;
public class SupervisedAsk {
private static class AskParam {
Props props;
Object message;
Timeout timeout;
AskParam(Props props, Object message, Timeout timeout) {
this.props = props;
this.message = message;
this.timeout = timeout;
}
}
private static class AskTimeout {
}
public static class AskSupervisorCreator extends UntypedActor {
@Override
public void onReceive(Object message) throws Exception {
if (message instanceof AskParam) {
ActorRef supervisor = getContext().actorOf(
Props.apply(AskSupervisor.class));
supervisor.forward(message, getContext());
} else {
unhandled(message);
}
}
}
public static class AskSupervisor extends UntypedActor {
private ActorRef targetActor;
private ActorRef caller;
private AskParam askParam;
private Cancellable timeoutMessage;
@Override
public SupervisorStrategy supervisorStrategy() {
return new OneForOneStrategy(0, Duration.Zero(),
new Function<Throwable, Directive>() {
public Directive apply(Throwable cause) {
caller.tell(new Status.Failure(cause), self());
return SupervisorStrategy.stop();
}
});
}
@Override
public void onReceive(Object message) throws Exception {
if (message instanceof AskParam) {
askParam = (AskParam) message;
caller = getSender();
targetActor = getContext().actorOf(askParam.props);
getContext().watch(targetActor);
targetActor.forward(askParam.message, getContext());
Scheduler scheduler = getContext().system().scheduler();
timeoutMessage = scheduler.scheduleOnce(askParam.timeout.duration(),
self(), new AskTimeout(), context().dispatcher());
} else if (message instanceof Terminated) {
Throwable ex = new ActorKilledException("Target actor terminated.");
caller.tell(new Status.Failure(ex), self());
timeoutMessage.cancel();
getContext().stop(self());
} else if (message instanceof AskTimeout) {
Throwable ex = new TimeoutException("Target actor timed out after "
+ askParam.timeout.toString());
caller.tell(new Status.Failure(ex), self());
getContext().stop(self());
} else
unhandled(message);
}
}
public static Future<Object> askOf(ActorRef supervisorCreator, Props props,
Object message, Timeout timeout) {
AskParam param = new AskParam(props, message, timeout);
return Patterns.ask(supervisorCreator, param, timeout);
}
synchronized public static ActorRef createSupervisorCreator(
ActorRefFactory factory) {
return factory.actorOf(Props.apply(AskSupervisorCreator.class));
}
}

View file

@ -0,0 +1,28 @@
package docs.pattern;
import scala.concurrent.Await;
import scala.concurrent.Future;
import akka.actor.ActorRef;
import akka.actor.ActorRefFactory;
import akka.actor.Props;
import akka.actor.UntypedActor;
import akka.util.Timeout;
public class SupervisedAskSpec {
public Object execute(Class<? extends UntypedActor> someActor,
Object message, Timeout timeout, ActorRefFactory actorSystem)
throws Exception {
// example usage
try {
ActorRef supervisorCreator = SupervisedAsk
.createSupervisorCreator(actorSystem);
Future<Object> finished = SupervisedAsk.askOf(supervisorCreator,
Props.apply(someActor), message, timeout);
return Await.result(finished, timeout.duration());
} catch (Exception e) {
// exception propagated by supervision
throw e;
}
}
}

View file

@ -138,12 +138,7 @@ public class SerializationDocTestBase {
}
public Address getAddress() {
final ActorRefProvider provider = system.provider();
if (provider instanceof RemoteActorRefProvider) {
return ((RemoteActorRefProvider) provider).transport().defaultAddress();
} else {
throw new UnsupportedOperationException("need RemoteActorRefProvider");
}
return system.provider().getDefaultAddress();
}
}

View file

@ -16,7 +16,7 @@ import akka.zeromq.Subscribe;
import akka.zeromq.Unsubscribe;
//#import-unsub-topic-socket
//#import-pub-topic
import akka.zeromq.Frame;
import akka.util.ByteString;
import akka.zeromq.ZMQMessage;
//#import-pub-topic
@ -96,9 +96,12 @@ public class ZeromqDocTestBase {
byte[] payload = new byte[0];
//#pub-topic
pubSocket.tell(new ZMQMessage(new Frame("foo.bar"), new Frame(payload)), null);
pubSocket.tell(ZMQMessage.withFrames(ByteString.fromString("foo.bar"), ByteString.fromArray(payload)), null);
//#pub-topic
system.stop(subSocket);
system.stop(subTopicSocket);
//#high-watermark
ActorRef highWatermarkSocket = ZeroMQExtension.get(system).newRouterSocket(
new SocketOption[] { new Listener(listener),
@ -133,7 +136,7 @@ public class ZeromqDocTestBase {
private boolean checkZeroMQInstallation() {
try {
ZeroMQVersion v = ZeroMQExtension.get(system).version();
return (v.major() == 2 && v.minor() == 1);
return (v.major() >= 3 || (v.major() >= 2 && v.minor() >= 1));
} catch (LinkageError e) {
return false;
}
@ -195,7 +198,7 @@ public class ZeromqDocTestBase {
public void preStart() {
getContext().system().scheduler()
.schedule(Duration.create(1, "second"), Duration.create(1, "second"),
getSelf(), TICK, getContext().dispatcher());
getSelf(), TICK, getContext().dispatcher(), null);
}
@Override
@ -210,18 +213,23 @@ public class ZeromqDocTestBase {
long timestamp = System.currentTimeMillis();
// use akka SerializationExtension to convert to bytes
byte[] heapPayload = ser.serializerFor(Heap.class).toBinary(
new Heap(timestamp, currentHeap.getUsed(), currentHeap.getMax()));
ByteString heapTopic = ByteString.fromString("health.heap", "UTF-8");
ByteString heapPayload = ByteString.fromArray(
ser.serialize(
new Heap(timestamp,
currentHeap.getUsed(),
currentHeap.getMax())
).get());
// the first frame is the topic, second is the message
pubSocket.tell(new ZMQMessage(new Frame("health.heap"),
new Frame(heapPayload)), getSelf());
pubSocket.tell(ZMQMessage.withFrames(heapTopic, heapPayload), getSelf());
// use akka SerializationExtension to convert to bytes
byte[] loadPayload = ser.serializerFor(Load.class).toBinary(
new Load(timestamp, os.getSystemLoadAverage()));
ByteString loadTopic = ByteString.fromString("health.load", "UTF-8");
ByteString loadPayload = ByteString.fromArray(
ser.serialize(new Load(timestamp, os.getSystemLoadAverage())).get()
);
// the first frame is the topic, second is the message
pubSocket.tell(new ZMQMessage(new Frame("health.load"),
new Frame(loadPayload)), getSelf());
pubSocket.tell(ZMQMessage.withFrames(loadTopic, loadPayload), getSelf());
} else {
unhandled(message);
}
@ -245,13 +253,14 @@ public class ZeromqDocTestBase {
public void onReceive(Object message) {
if (message instanceof ZMQMessage) {
ZMQMessage m = (ZMQMessage) message;
String topic = m.frame(0).utf8String();
// the first frame is the topic, second is the message
if (m.firstFrameAsString().equals("health.heap")) {
Heap heap = (Heap) ser.serializerFor(Heap.class).fromBinary(m.payload(1));
if ("health.heap".equals(topic)) {
Heap heap = ser.deserialize(m.frame(1).toArray(), Heap.class).get();
log.info("Used heap {} bytes, at {}", heap.used,
timestampFormat.format(new Date(heap.timestamp)));
} else if (m.firstFrameAsString().equals("health.load")) {
Load load = (Load) ser.serializerFor(Load.class).fromBinary(m.payload(1));
} else if ("health.load".equals(topic)) {
Load load = ser.deserialize(m.frame(1).toArray(), Load.class).get();
log.info("Load average {}, at {}", load.loadAverage,
timestampFormat.format(new Date(load.timestamp)));
}
@ -279,9 +288,10 @@ public class ZeromqDocTestBase {
public void onReceive(Object message) {
if (message instanceof ZMQMessage) {
ZMQMessage m = (ZMQMessage) message;
String topic = m.frame(0).utf8String();
// the first frame is the topic, second is the message
if (m.firstFrameAsString().equals("health.heap")) {
Heap heap = (Heap) ser.serializerFor(Heap.class).fromBinary(m.payload(1));
if ("health.heap".equals(topic)) {
Heap heap = ser.<Heap>deserialize(m.frame(1).toArray(), Heap.class).get();
if (((double) heap.used / heap.max) > 0.9) {
count += 1;
} else {

View file

@ -185,7 +185,7 @@ at runtime::
system.eventStream.setLogLevel(Logging.DebugLevel());
This means that log events for a level which will not be logged are not
This means that log events for a level which will not be logged are
typically not dispatched at all (unless manual subscriptions to the respective
event class have been done)

View file

@ -24,9 +24,6 @@ sample as it is easy to follow the log output to understand what is happening in
fault-tolerance-sample
.. includecode:: code/docs/actor/japi/FaultHandlingDocSample.java#all
:exclude: imports,messages,dummydb
Creating a Supervisor Strategy
------------------------------

View file

@ -1,4 +1,3 @@
.. _howto-java:
######################
@ -48,6 +47,43 @@ and schedule the initial message send again.
.. includecode:: code/docs/pattern/SchedulerPatternTest.java#schedule-receive
Single-Use Actor Trees with High-Level Error Reporting
======================================================
*Contributed by: Rick Latrine*
A nice way to enter the actor world from java is the use of Patterns.ask().
This method starts a temporary actor to forward the message and collect the result from the actor to be "asked".
In case of errors within the asked actor the default supervision handling will take over.
The caller of Patterns.ask() will *not* be notified.
If that caller is interested in such an exception, he must make sure that the asked actor replies with Status.Failure(Throwable).
Behind the asked actor a complex actor hierarchy might be spawned to accomplish asynchronous work.
Then supervision is the established way to control error handling.
Unfortunately the asked actor must know about supervision and must catch the exceptions.
Such an actor is unlikely to be reused in a different actor hierarchy and contains crippled try/catch blocks.
This pattern provides a way to encapsulate supervision and error propagation to the temporary actor.
Finally the promise returned by Patterns.ask() is fulfilled as a failure, including the exception.
Let's have a look at the example code:
.. includecode:: code/docs/pattern/SupervisedAsk.java
In the askOf method the SupervisorCreator is sent the user message.
The SupervisorCreator creates a SupervisorActor and forwards the message.
This prevents the actor system from overloading due to actor creations.
The SupervisorActor is responsible to create the user actor, forwards the message, handles actor termination and supervision.
Additionally the SupervisorActor stops the user actor if execution time expired.
In case of an exception the supervisor tells the temporary actor which exception was thrown.
Afterwards the actor hierarchy is stopped.
Finally we are able to execute an actor and receive the results or exceptions.
.. includecode:: code/docs/pattern/SupervisedAskSpec.java
Template Pattern
================

View file

@ -19,11 +19,7 @@ Put your application jar in the ``deploy`` directory to have it automatically
loaded.
To start the kernel use the scripts in the ``bin`` directory, passing the boot
classes for your application.
There is a simple example of an application setup for running with the
microkernel included in the akka download. This can be run with the following
command (on a unix-based system):
classes for your application. Example command (on a unix-based system):
.. code-block:: none

View file

@ -122,6 +122,15 @@ actor systems has to have a JAR containing the class.
object, which in most cases is not serializable. It is best to make a static
inner class which implements :class:`UntypedActorFactory`.
.. note::
You can use asterisks as wildcard matches for the actor path sections, so you could specify:
``/*/sampleActor`` and that would match all ``sampleActor`` on that level in the hierarchy.
You can also use wildcard in the last position to match all actors at a certain level:
``/someParent/*``. Non-wildcard matches always have higher priority to match than wildcards, so:
``/foo/bar`` is considered **more specific** than ``/foo/*`` and only the highest priority match is used.
Please note that it **cannot** be used to partially match section, like this: ``/foo*/bar``, ``/f*o/bar`` etc.
.. warning::
*Caveat:* Remote deployment ties both systems together in a tight fashion,

View file

@ -66,7 +66,7 @@ In addition to being able to supply looked-up remote actors as routees, you can
make the router deploy its created children on a set of remote hosts; this will
be done in round-robin fashion. In order to do that, wrap the router
configuration in a :class:`RemoteRouterConfig`, attaching the remote addresses of
the nodes to deploy to. Naturally, this requires your to include the
the nodes to deploy to. Naturally, this requires you to include the
``akka-remote`` module on your classpath:
.. includecode:: code/docs/jrouting/RouterViaProgramExample.java#remoteRoutees
@ -114,7 +114,7 @@ Routers vs. Supervision
^^^^^^^^^^^^^^^^^^^^^^^
As explained in the previous section, routers create new actor instances as
children of the “head” router, who therefor also is their supervisor. The
children of the “head” router, who therefore also is their supervisor. The
supervisor strategy of this actor can be configured by means of the
:meth:`RouterConfig.supervisorStrategy` property, which is supported for all
built-in router types. It defaults to “always escalate”, which leads to the
@ -434,7 +434,7 @@ Configured Custom Router
It is possible to define configuration properties for custom routers. In the ``router`` property of the deployment
configuration you define the fully qualified class name of the router class. The router class must extend
``akka.routing.CustomRouterConfig`` and and have constructor with ``com.typesafe.config.Config`` parameter.
``akka.routing.CustomRouterConfig`` and have constructor with one ``com.typesafe.config.Config`` parameter.
The deployment section of the configuration is passed to the constructor.
Custom Resizer

View file

@ -149,16 +149,12 @@ concrete address handy you can create a dummy one for the right protocol using
``new Address(protocol, "", "", 0)`` (assuming that the actual transport used is as
lenient as Akkas RemoteActorRefProvider).
There is a possible simplification available if you are just using the default
:class:`NettyRemoteTransport` with the :meth:`RemoteActorRefProvider`, which is
enabled by the fact that this combination has just a single remote address:
There is also a default remote address which is the one used by cluster support
(and typical systems have just this one); you can get it like this:
.. includecode:: code/docs/serialization/SerializationDocTestBase.java
:include: external-address-default
This solution has to be adapted once other providers are used (like the planned
extensions for clustering).
Deep serialization of Actors
----------------------------

View file

@ -127,10 +127,11 @@ UntypedActor API
The :class:`UntypedActor` class defines only one abstract method, the above mentioned
:meth:`onReceive(Object message)`, which implements the behavior of the actor.
If the current actor behavior does not match a received message,
:meth:`unhandled` is called, which by default publishes a ``new
If the current actor behavior does not match a received message, it's recommended that
you call the :meth:`unhandled` method, which by default publishes a ``new
akka.actor.UnhandledMessage(message, sender, recipient)`` on the actor systems
event stream.
event stream (set configuration item ``akka.actor.debug.unhandled`` to ``on``
to have them converted into actual Debug messages).
In addition, it offers:

View file

@ -96,7 +96,8 @@ added in concrete subclass like this:
To use ``DurableMailboxDocSpec`` add this dependency::
"com.typesafe.akka" %% "akka-mailboxes-common" % "@version@" classifier "test" @crossString@
"com.typesafe.akka" %% "akka-mailboxes-common" %
"@version@" classifier "test" @crossString@
For more inspiration you can look at the old implementations based on Redis, MongoDB, Beanstalk,
and ZooKeeper, which can be found in Akka git repository tag

View file

@ -73,7 +73,8 @@ Make sure that you add the repository to the sbt resolvers::
Define the library dependencies with the timestamp as version. For example::
libraryDependencies += "com.typesafe.akka" % "akka-remote_@binVersion@" % "2.1-20121016-001042"
libraryDependencies += "com.typesafe.akka" % "akka-remote_@binVersion@" %
"2.1-20121016-001042"
maven definition of snapshot repository
---------------------------------------

View file

@ -4,27 +4,35 @@
Migration Guide 2.0.x to 2.1.x
################################
The 2.1 release contains several structural changes that require some
simple, mechanical source-level changes in client code. Several things have
been moved to Scala standard library, such as ``Future``, and some package
names have been changed in Remoting and Durable Mailboxes.
Some parts of the 2.0 API have changed in the Akka 2.1 release. This guide lists the the changes and
explains what you will need to do to upgrade your program to work with Akka 2.1.
When migrating from 1.3.x to 2.1.x you should first follow the instructions for
migrating `1.3.x to 2.0.x <http://doc.akka.io/docs/akka/2.0.3/project/migration-guide-1.3.x-2.0.x.html>`_.
Migrating from Akka 2.0.x to Akka 2.1.x is relatively straightforward. In Akka 2.1 the API has
undergone some basic housekeeping, for example some package names have changed, but otherwise usage
is largely unchanged. User programs will generally only need simple, mechanical changes in order to
work with Akka 2.1.
If you are migrating from Akka 1.3.x you will need to follow the instructions for
`migrating from Akka 1.3.x to 2.0.x <http://doc.akka.io/docs/akka/2.0.3/project/migration-guide-1.3.x-2.0.x.html>`_
before following the instructions in this guide.
Scala Version
=============
Akka 2.1 uses a new version of Scala.
Change your project build and dependencies to Scala version ``@scalaVersion@``.
Config Dependency
=================
`Typesafe config <https://github.com/typesafehub/config>`_ library is a normal
dependency of akka-actor and it is no longer embedded in ``akka-actor.jar``.
If your are using a build tool with dependency resolution, such as sbt or maven you
will not notice the difference, but if you have manually constructed classpaths
you need to add `config-1.0.0.jar <http://mirrors.ibiblio.org/maven2/com/typesafe/config/1.0.0/>`_.
Akka's configuration system has graduated from Akka to become the `Typesafe config
<https://github.com/typesafehub/config>`_ project. The configuration system was previously embedded
within ``akka-actor.jar``, now it is specified as a dependency of ``akka-actor.jar``.
If your are using a build tool with automatic dependency resolution, such as sbt or Maven, then you
will not notice a difference. Otherwise you will need to ensure that
`config-1.0.0.jar <http://mirrors.ibiblio.org/maven2/com/typesafe/config/1.0.0/>`_
is present on your classpath.
Pieces Moved to Scala Standard Library
======================================
@ -48,7 +56,8 @@ Search Replace with
Scheduler Dispatcher
====================
The ``ExecutionContext`` to use for running scheduled tasks must be specified.
The ``ExecutionContext`` to use for running scheduled tasks must now be specified.
You can use an Akka ``Dispatcher`` for this purpose.
Scala:
@ -66,8 +75,9 @@ Java:
::
// Use this Actors' Dispatcher as ExecutionContext
getContext().system().scheduler().scheduleOnce(Duration.create(10, TimeUnit.SECONDS)",
getSelf(), new Reconnect(), getContext().getDispatcher());
getContext().system().scheduler().scheduleOnce(Duration.create(
10, TimeUnit.SECONDS), getSelf(), new Reconnect(),
getContext().getDispatcher());
// Use ActorSystem's default Dispatcher as ExecutionContext
system.scheduler().scheduleOnce(Duration.create(50, TimeUnit.MILLISECONDS),
@ -79,7 +89,7 @@ Java:
}, system.dispatcher());
API Changes of Future - Scala
API Changes to Future - Scala
=============================
v2.0::
@ -105,7 +115,7 @@ v2.1::
API Changes of Future - Java
API Changes to Future - Java
============================
v2.0::
@ -181,17 +191,17 @@ v2.1::
}
}, ec);
API changes of DynamicAccess
API changes to DynamicAccess
============================
All methods with scala.Either[Throwable, X] have been changed to used scala.util.Try[X].
All methods with scala.Either[Throwable, X] have been changed to use scala.util.Try[X].
DynamicAccess.withErrorHandling has been removed since scala.util.Try now fulfills that role.
API changes of Serialization
API changes to Serialization
============================
All methods with scala.Either[Throwable, X] have been changed to used scala.util.Try[X].
All methods with scala.Either[Throwable, X] have been changed to use scala.util.Try[X].
Empty Props
===========
@ -230,14 +240,14 @@ v2.1 Scala::
Failing Send
============
When failing to send to a remote actor or actor with bounded or durable mailbox the message will
silently be delivered to ``ActorSystem.deadletters`` instead of throwing an exception.
When failing to send to a remote actor or an actor with a bounded or durable mailbox the message will
now be silently delivered to ``ActorSystem.deadletters`` instead of throwing an exception.
Graceful Stop Exception
=======================
If the target actor of ``akka.pattern.gracefulStop`` isn't terminated within the
timeout the ``Future`` is completed with failure ``akka.pattern.AskTimeoutException``.
timeout then the ``Future`` is completed with a failure of ``akka.pattern.AskTimeoutException``.
In 2.0 it was ``akka.actor.ActorTimeoutException``.
getInstance for Singletons - Java
@ -275,23 +285,23 @@ v2.1::
log-remote-lifecycle-events
===========================
Default value of akka.remote.log-remote-lifecycle-events has changed to **on**.
If you don't want these in the log you need to add this to your configuration::
The default value of akka.remote.log-remote-lifecycle-events has changed to **on**.
If you don't want these events in the log then you need to add this to your configuration::
akka.remote.log-remote-lifecycle-events = off
Stash postStop
==============
Both Actors and UntypedActors using ``Stash`` now overrides postStop to make sure that
stashed messages are put into the dead letters when the actor stops, make sure you call
Both Actors and UntypedActors using ``Stash`` now override postStop to make sure that
stashed messages are put into the dead letters when the actor stops. Make sure you call
super.postStop if you override it.
Forward of Terminated message
=============================
Forwarding Terminated messages
==============================
Forward of ``Terminated`` message is no longer supported. Instead, if you forward
``Terminated`` you should send the information in you own message.
Forwarding ``Terminated`` messages is no longer supported. Instead, if you forward
``Terminated`` you should send the information in your own message.
v2.0::
@ -312,12 +322,12 @@ v2.1::
}
Custom Router or Resizer
========================
Custom Routers and Resizers
===========================
The API of ``RouterConfig``, ``RouteeProvider`` and ``Resizer`` has been
cleaned up. If you use these to build your own router functionality the
compiler will tell you you to do some adjustments.
compiler will tell you if you need to make adjustments.
v2.0::
@ -362,19 +372,20 @@ v2.1::
Duration and Timeout
====================
The Duration class in the scala library is an improved version of the previous
:class:`akka.util.Duration`. Among others it keeps the static type of
:class:`FiniteDuration` more consistently, which has been used to tighten APIs.
The advantage is that instead of runtime exceptions youll get compiler errors
telling you if you try to pass a possibly non-finite duration where it does not
belong.
The :class:`akka.util.Duration` class has been moved into the Scala library under
the ``scala.concurrent.duration`` package. Several changes have been made to tighten
up the duration and timeout API.
:class:`FiniteDuration` is now used more consistently throught the API.
The advantage is that if you try to pass a possibly non-finite duration where
it does not belong youll get compile errors instead of runtime exceptions.
The main source incompatibility is that you may have to change the declared
type of fields from ``Duration`` to ``FiniteDuration`` (factory methods already
return the more precise type wherever possible).
Another change is that ``Duration.parse`` was not accepted by the scala-library
maintainers, use ``Duration.create`` instead.
Another change is that ``Duration.parse`` was not accepted by the Scala library
maintainers; use ``Duration.create`` instead.
v2.0::
@ -392,38 +403,49 @@ Package Name Changes in Remoting
The package name of all classes in the ``akka-remote.jar`` artifact now starts with ``akka.remote``.
This has been done to enable OSGi bundles that don't have conflicting package names.
Change the following import statements. Please note that the serializers are often referenced from configuration.
Change the following import statements. Please note that serializers are often referenced from
configuration files.
Search -> Replace with::
akka.routing.RemoteRouterConfig ->
akka.remote.routing.RemoteRouterConfig
akka.serialization.ProtobufSerializer ->
akka.remote.serialization.ProtobufSerializer
akka.serialization.DaemonMsgCreateSerializer ->
akka.remote.serialization.DaemonMsgCreateSerializer
================================================ =======================================================
Search Replace with
================================================ =======================================================
``akka.routing.RemoteRouterConfig`` ``akka.remote.routing.RemoteRouterConfig``
``akka.serialization.ProtobufSerializer`` ``akka.remote.serialization.ProtobufSerializer``
``akka.serialization.DaemonMsgCreateSerializer`` ``akka.remote.serialization.DaemonMsgCreateSerializer``
================================================ =======================================================
Package Name Changes in Durable Mailboxes
=========================================
The package name of all classes in the ``akka-file-mailbox.jar`` artifact now starts with ``akka.actor.mailbox.filebased``.
The package names of all classes in the ``akka-file-mailbox.jar`` artifact now start with ``akka.actor.mailbox.filebased``.
This has been done to enable OSGi bundles that don't have conflicting package names.
Change the following import statements. Please note that the ``FileBasedMailboxType`` is often referenced from configuration.
================================================ =========================================================
Search Replace with
================================================ =========================================================
``akka.actor.mailbox.FileBasedMailboxType`` ``akka.actor.mailbox.filebased.FileBasedMailboxType``
``akka.actor.mailbox.FileBasedMailboxSettings`` ``akka.actor.mailbox.filebased.FileBasedMailboxSettings``
``akka.actor.mailbox.FileBasedMessageQueue`` ``akka.actor.mailbox.filebased.FileBasedMessageQueue``
``akka.actor.mailbox.filequeue.*`` ``akka.actor.mailbox.filebased.filequeue.*``
================================================ =========================================================
Search -> Replace with::
akka.actor.mailbox.FileBasedMailboxType ->
akka.actor.mailbox.filebased.FileBasedMailboxType
akka.actor.mailbox.FileBasedMailboxSettings ->
akka.actor.mailbox.filebased.FileBasedMailboxSettings
akka.actor.mailbox.FileBasedMessageQueue ->
akka.actor.mailbox.filebased.FileBasedMessageQueue
akka.actor.mailbox.filequeue.* ->
akka.actor.mailbox.filebased.filequeue.*
Actor Receive Timeout
=====================
The API for setting and querying the receive timeout has been made more
consisten in always taking and returning a ``Duration``, the wrapping in
consistent in always taking and returning a ``Duration``; the wrapping in
``Option`` has been removed.
(Samples for Java, Scala sources are affected in exactly the same way.)
@ -445,7 +467,7 @@ v2.1::
ConsistentHash
==============
``akka.routing.ConsistentHash`` has been changed to an immutable data structure.
``akka.routing.ConsistentHash`` has been changed into an immutable data structure.
v2.0::

View file

@ -23,4 +23,33 @@ Search Replace with
==================================== ====================================
If you need to convert from Java to ``scala.collection.immutable.Seq`` or ``scala.collection.immutable.Iterable`` you should use ``akka.japi.Util.immutableSeq(…)``,
and if you need to convert from Scala you can simply switch to using immutable collections yourself or use the ``to[immutable.<collection-type>]`` method.
and if you need to convert from Scala you can simply switch to using immutable collections yourself or use the ``to[immutable.<collection-type>]`` method.
API changes to FSM and TestFSMRef
=================================
The ``timerActive_?`` method has been deprecated in both the ``FSM`` trait and the ``TestFSMRef``
class. You should now use the ``isTimerActive`` method instead. The old method will remain
throughout 2.2.x. It will be removed in Akka 2.3.
ThreadPoolConfigBuilder
=======================
``akka.dispatch.ThreadPoolConfigBuilder`` companion object has been removed,
and with it the ``conf_?`` method that was essentially only a type-inferencer aid for creation
of optional transformations on ``ThreadPoolConfigBuilder``.
Instead use: ``option.map(o => (t: ThreadPoolConfigBuilder) => t.op(o))``.
Scheduler
=========
Akka's ``Scheduler`` has been augmented to also include a ``sender`` when scheduling to send messages, this should work Out-Of-The-Box for Scala users,
but for Java Users you will need to manually provide the ``sender`` as usual use ``null`` to designate "no sender" which will behave just as before the change.
ZeroMQ ByteString
=================
``akka.zeromq.Frame`` and the use of ``Seq[Byte]`` in the API has been removed and is replaced by ``akka.util.ByteString``.
``ZMQMessage.firstFrameAsString`` has been removed, please use ``ZMQMessage.frames`` or ``ZMQMessage.frame(int)`` to access the frames.

View file

@ -234,8 +234,8 @@ If the current actor behavior does not match a received message,
:meth:`unhandled` is called, which by default publishes an
``akka.actor.UnhandledMessage(message, sender, recipient)`` on the actor
systems event stream (set configuration item
``akka.event-handler-startup-timeout`` to ``true`` to have them converted into
actual Debug messages)
``akka.actor.debug.unhandled`` to ``on`` to have them converted into
actual Debug messages).
In addition, it offers:

View file

@ -384,6 +384,8 @@ URI options
The following URI options are supported:
.. tabularcolumns:: |l|l|l|L|
+--------------+----------+---------+-------------------------------------------+
| Name | Type | Default | Description |
+==============+==========+=========+===========================================+

View file

@ -189,6 +189,15 @@ class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit {
}
//#fsm-code-elided
"demonstrate NullFunction" in {
class A extends Actor with FSM[Int, Null] {
val SomeState = 0
//#NullFunction
when(SomeState)(FSM.NullFunction)
//#NullFunction
}
}
"batch correctly" in {
val buncher = system.actorOf(Props(new Buncher))
buncher ! SetTarget(testActor)

View file

@ -417,7 +417,7 @@ class FutureDocSpec extends AkkaSpec {
val delayed = after(200 millis, using = system.scheduler)(Future.failed(
new IllegalStateException("OHNOES")))
val future = Future { Thread.sleep(1000); "foo" }
val result = future either delayed
val result = Future firstCompletedOf Seq(future, delayed)
//#after
intercept[IllegalStateException] { Await.result(result, 2 second) }
}

View file

@ -2,19 +2,6 @@
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
//#extract-transport
package object akka {
// needs to be inside the akka package because accessing unsupported API !
def transportOf(system: actor.ExtendedActorSystem): remote.RemoteTransport =
system.provider match {
case r: remote.RemoteActorRefProvider r.transport
case _
throw new UnsupportedOperationException(
"this method requires the RemoteActorRefProvider to be configured")
}
}
//#extract-transport
package docs.serialization {
import org.scalatest.matchers.MustMatchers
@ -216,7 +203,7 @@ package docs.serialization {
object ExternalAddress extends ExtensionKey[ExternalAddressExt]
class ExternalAddressExt(system: ExtendedActorSystem) extends Extension {
def addressForAkka: Address = akka.transportOf(system).defaultAddress
def addressForAkka: Address = system.provider.getDefaultAddress
}
def serializeAkkaDefault(ref: ActorRef): String =

View file

@ -110,11 +110,11 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
fsm.setState(stateName = 1)
assert(fsm.stateName == 1)
assert(fsm.timerActive_?("test") == false)
assert(fsm.isTimerActive("test") == false)
fsm.setTimer("test", 12, 10 millis, true)
assert(fsm.timerActive_?("test") == true)
assert(fsm.isTimerActive("test") == true)
fsm.cancelTimer("test")
assert(fsm.timerActive_?("test") == false)
assert(fsm.isTimerActive("test") == false)
//#test-fsm-ref
}
@ -232,7 +232,7 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
//#test-probe-forward
}
"demonstrate " in {
"demonstrate calling thread dispatcher" in {
//#calling-thread-dispatcher
import akka.testkit.CallingThreadDispatcher
val ref = system.actorOf(Props[MyActor].withDispatcher(CallingThreadDispatcher.Id))

View file

@ -6,8 +6,8 @@ package docs.zeromq
import language.postfixOps
import scala.concurrent.duration._
import scala.collection.immutable
import akka.actor.{ Actor, Props }
import akka.util.ByteString
import akka.testkit._
import akka.zeromq.{ ZeroMQVersion, ZeroMQExtension, SocketType, Bind }
import java.text.SimpleDateFormat
@ -29,7 +29,8 @@ object ZeromqDocSpec {
class HealthProbe extends Actor {
val pubSocket = ZeroMQExtension(context.system).newSocket(SocketType.Pub, Bind("tcp://127.0.0.1:1235"))
val pubSocket = ZeroMQExtension(context.system).newSocket(SocketType.Pub,
Bind("tcp://127.0.0.1:1235"))
val memory = ManagementFactory.getMemoryMXBean
val os = ManagementFactory.getOperatingSystemMXBean
val ser = SerializationExtension(context.system)
@ -52,12 +53,12 @@ object ZeromqDocSpec {
val heapPayload = ser.serialize(Heap(timestamp, currentHeap.getUsed,
currentHeap.getMax)).get
// the first frame is the topic, second is the message
pubSocket ! ZMQMessage(immutable.Seq(Frame("health.heap"), Frame(heapPayload)))
pubSocket ! ZMQMessage(ByteString("health.heap"), ByteString(heapPayload))
// use akka SerializationExtension to convert to bytes
val loadPayload = ser.serialize(Load(timestamp, os.getSystemLoadAverage)).get
// the first frame is the topic, second is the message
pubSocket ! ZMQMessage(immutable.Seq(Frame("health.load"), Frame(loadPayload)))
pubSocket ! ZMQMessage(ByteString("health.load"), ByteString(loadPayload))
}
}
//#health
@ -72,14 +73,14 @@ object ZeromqDocSpec {
def receive = {
// the first frame is the topic, second is the message
case m: ZMQMessage if m.firstFrameAsString == "health.heap"
val Heap(timestamp, used, max) = ser.deserialize(m.payload(1),
case m: ZMQMessage if m.frames(0).utf8String == "health.heap"
val Heap(timestamp, used, max) = ser.deserialize(m.frames(1).toArray,
classOf[Heap]).get
log.info("Used heap {} bytes, at {}", used,
timestampFormat.format(new Date(timestamp)))
case m: ZMQMessage if m.firstFrameAsString == "health.load"
val Load(timestamp, loadAverage) = ser.deserialize(m.payload(1),
case m: ZMQMessage if m.frames(0).utf8String == "health.load"
val Load(timestamp, loadAverage) = ser.deserialize(m.frames(1).toArray,
classOf[Load]).get
log.info("Load average {}, at {}", loadAverage,
timestampFormat.format(new Date(timestamp)))
@ -97,9 +98,8 @@ object ZeromqDocSpec {
def receive = {
// the first frame is the topic, second is the message
case m: ZMQMessage if m.firstFrameAsString == "health.heap"
val Heap(timestamp, used, max) = ser.deserialize(m.payload(1),
classOf[Heap]).get
case m: ZMQMessage if m.frames(0).utf8String == "health.heap"
val Heap(timestamp, used, max) = ser.deserialize(m.frames(1).toArray, classOf[Heap]).get
if ((used.toDouble / max) > 0.9) count += 1
else count = 0
if (count > 10) log.warning("Need more memory, using {} %",
@ -146,7 +146,7 @@ class ZeromqDocSpec extends AkkaSpec("akka.loglevel=INFO") {
val payload = Array.empty[Byte]
//#pub-topic
pubSocket ! ZMQMessage(Frame("foo.bar"), Frame(payload))
pubSocket ! ZMQMessage(ByteString("foo.bar"), ByteString(payload))
//#pub-topic
system.stop(subSocket)
@ -187,8 +187,9 @@ class ZeromqDocSpec extends AkkaSpec("akka.loglevel=INFO") {
def checkZeroMQInstallation() = try {
ZeroMQExtension(system).version match {
case ZeroMQVersion(2, 1, _) Unit
case version pending
case ZeroMQVersion(2, x, _) if x >= 1 Unit
case ZeroMQVersion(y, _, _) if y >= 3 Unit
case version pending
}
} catch {
case e: LinkageError pending

View file

@ -24,9 +24,6 @@ sample as it is easy to follow the log output to understand what is happening in
fault-tolerance-sample
.. includecode:: code/docs/actor/FaultHandlingDocSample.scala#all
:exclude: imports,messages,dummydb
Creating a Supervisor Strategy
------------------------------

View file

@ -179,6 +179,18 @@ demonstrated below:
The :class:`Event(msg: Any, data: D)` case class is parameterized with the data
type held by the FSM for convenient pattern matching.
.. warning::
It is required that you define handlers for each of the possible FSM states,
otherwise there will be failures when trying to switch to undeclared states.
It is recommended practice to declare the states as objects extending a
sealed trait and then verify that there is a ``when`` clause for each of the
states. If you want to leave the handling of a state “unhandled” (more below),
it still needs to be declared like this:
.. includecode:: code/docs/actor/FSMDocSpec.scala#NullFunction
Defining the Initial State
--------------------------
@ -359,7 +371,7 @@ which is guaranteed to work immediately, meaning that the scheduled message
will not be processed after this call even if the timer already fired and
queued it. The status of any timer may be inquired with
:func:`timerActive_?(name)`
:func:`isTimerActive(name)`
These named timers complement state timeouts because they are not affected by
intervening reception of other messages.

View file

@ -138,9 +138,9 @@ Receiving messages from the ``IOManager``:
IO.Iteratee
^^^^^^^^^^^
Included with Akka's IO support is a basic implementation of ``Iteratee``\s. ``Iteratee``\s are an effective way of handling a stream of data without needing to wait for all the data to arrive. This is especially useful when dealing with non blocking IO since we will usually receive data in chunks which may not include enough information to process, or it may contain much more data then we currently need.
Included with Akka's IO support is a basic implementation of ``Iteratee``\s. ``Iteratee``\s are an effective way of handling a stream of data without needing to wait for all the data to arrive. This is especially useful when dealing with non blocking IO since we will usually receive data in chunks which may not include enough information to process, or it may contain much more data than we currently need.
This ``Iteratee`` implementation is much more basic then what is usually found. There is only support for ``ByteString`` input, and enumerators aren't used. The reason for this limited implementation is to reduce the amount of explicit type signatures needed and to keep things simple. It is important to note that Akka's ``Iteratee``\s are completely optional, incoming data can be handled in any way, including other ``Iteratee`` libraries.
This ``Iteratee`` implementation is much more basic than what is usually found. There is only support for ``ByteString`` input, and enumerators aren't used. The reason for this limited implementation is to reduce the amount of explicit type signatures needed and to keep things simple. It is important to note that Akka's ``Iteratee``\s are completely optional, incoming data can be handled in any way, including other ``Iteratee`` libraries.
``Iteratee``\s work by processing the data that it is given and returning either the result (with any unused input) or a continuation if more input is needed. They are monadic, so methods like ``flatMap`` can be used to pass the result of an ``Iteratee`` to another.
@ -204,7 +204,7 @@ Following the path we read in the query (if it exists):
.. includecode:: code/docs/io/HTTPServer.scala
:include: read-query
It is much simpler then reading the path since we aren't doing any parsing of the query since there is no standard format of the query string.
It is much simpler than reading the path since we aren't doing any parsing of the query since there is no standard format of the query string.
Both the path and query used the ``readUriPart`` ``Iteratee``, which is next:

View file

@ -19,11 +19,7 @@ Put your application jar in the ``deploy`` directory to have it automatically
loaded.
To start the kernel use the scripts in the ``bin`` directory, passing the boot
classes for your application.
There is a simple example of an application setup for running with the
microkernel included in the akka download. This can be run with the following
command (on a unix-based system):
classes for your application. Example command (on a unix-based system):
.. code-block:: none

View file

@ -129,6 +129,15 @@ actor systems has to have a JAR containing the class.
most cases is not serializable. It is best to create a factory method in the
companion object of the actors class.
.. note::
You can use asterisks as wildcard matches for the actor paths, so you could specify:
``/*/sampleActor`` and that would match all ``sampleActor`` on that level in the hierarchy.
You can also use wildcard in the last position to match all actors at a certain level:
``/someParent/*``. Non-wildcard matches always have higher priority to match than wildcards, so:
``/foo/bar`` is considered **more specific** than ``/foo/*`` and only the highest priority match is used.
Please note that it **cannot** be used to partially match section, like this: ``/foo*/bar``, ``/f*o/bar`` etc.
.. warning::
*Caveat:* Remote deployment ties both systems together in a tight fashion,

View file

@ -66,7 +66,7 @@ In addition to being able to supply looked-up remote actors as routees, you can
make the router deploy its created children on a set of remote hosts; this will
be done in round-robin fashion. In order to do that, wrap the router
configuration in a :class:`RemoteRouterConfig`, attaching the remote addresses of
the nodes to deploy to. Naturally, this requires your to include the
the nodes to deploy to. Naturally, this requires you to include the
``akka-remote`` module on your classpath:
.. includecode:: code/docs/routing/RouterViaProgramExample.scala#remoteRoutees
@ -430,7 +430,7 @@ Configured Custom Router
It is possible to define configuration properties for custom routers. In the ``router`` property of the deployment
configuration you define the fully qualified class name of the router class. The router class must extend
``akka.routing.RouterConfig`` and and have constructor with ``com.typesafe.config.Config`` parameter.
``akka.routing.RouterConfig`` and have constructor with one ``com.typesafe.config.Config`` parameter.
The deployment section of the configuration is passed to the constructor.
Custom Resizer

View file

@ -138,24 +138,12 @@ concrete address handy you can create a dummy one for the right protocol using
``Address(protocol, "", "", 0)`` (assuming that the actual transport used is as
lenient as Akkas RemoteActorRefProvider).
There is a possible simplification available if you are just using the default
:class:`NettyRemoteTransport` with the :meth:`RemoteActorRefProvider`, which is
enabled by the fact that this combination has just a single remote address.
This approach relies on internal API, which means that it is not guaranteed to
be supported in future versions. To make this caveat more obvious, some bridge
code in the ``akka`` package is required to make it work:
.. includecode:: code/docs/serialization/SerializationDocSpec.scala
:include: extract-transport
And with this, the address extraction goes like this:
There is also a default remote address which is the one used by cluster support
(and typical systems have just this one); you can get it like this:
.. includecode:: code/docs/serialization/SerializationDocSpec.scala
:include: external-address-default
This solution has to be adapted once other providers are used (like the planned
extensions for clustering).
Deep serialization of Actors
----------------------------