improve AbstractActor, #21717

* Receive class that wraps PartialFunction, to avoid
  scary scala types
* move AbstractActorContext to AbstractActor.ActorContext
* converting docs, many, many UntypedActor
* removing UntypedActor docs
* add unit test for ReceiveBuilder
* MiMa filters
* consistent use of getContext(), self(), sender()
* rename cross references
* migration guide
* skip samples for now
* improve match type safetyi, add matchUnchecked
  * the `? extends P` caused code like this to compile:
    `match(String.class, (Integer i) -> {})`
  * added matchUnchecked, since it can still be useful (um, convenient)
    to be able to do:
    `matchUnchecked(List.class, (List<String> list) -> {})`
* eleminate some scala.Option
  * preRestart
  * findChild
  * ActorIdentity.getActorRef
This commit is contained in:
Patrik Nordwall 2016-12-13 10:59:29 +01:00
parent 3617fe8b41
commit 4bd6b7aab1
157 changed files with 3290 additions and 8882 deletions

View file

@ -12,19 +12,23 @@ import akka.testkit.TestEvent;
import com.typesafe.config.Config;
import com.typesafe.config.ConfigFactory;
import docs.AbstractJavaTest;
import docs.actor.ActorDocTest.FirstActor;
import scala.PartialFunction;
import scala.runtime.BoxedUnit;
import static docs.actorlambda.Messages.Swap.Swap;
import static docs.actorlambda.Messages.*;
import static akka.japi.Util.immutableSeq;
import akka.actor.CoordinatedShutdown;
import static akka.pattern.PatternsCS.ask;
import akka.util.Timeout;
import akka.Done;
import java.util.concurrent.CompletionStage;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.Optional;
import java.util.concurrent.TimeUnit;
import akka.testkit.TestActors;
import akka.dispatch.Mapper;
import akka.dispatch.Futures;
import akka.util.Timeout;
import akka.testkit.JavaTestKit;
import org.junit.AfterClass;
@ -44,13 +48,25 @@ import akka.actor.ActorIdentity;
import akka.actor.ActorSelection;
import akka.actor.Identify;
//#import-identify
//#import-graceFulStop
//#import-ask
import static akka.pattern.Patterns.ask;
import static akka.pattern.Patterns.pipe;
import akka.dispatch.Futures;
import akka.dispatch.Mapper;
import akka.util.Timeout;
//#import-ask
//#import-gracefulStop
import akka.pattern.AskTimeoutException;
import scala.concurrent.Await;
import scala.concurrent.duration.Duration;
//#import-ask
import scala.concurrent.Future;
import static akka.pattern.Patterns.gracefulStop;
//#import-graceFulStop
//#import-ask
//#import-gracefulStop
//#import-terminated
import akka.actor.Terminated;
//#import-terminated
public class ActorDocTest extends AbstractJavaTest {
@ -71,49 +87,111 @@ public class ActorDocTest extends AbstractJavaTest {
@AfterClass
public static void afterClass() throws Exception {
Await.ready(system.terminate(), Duration.create("5 seconds"));
Await.ready(system.terminate(), Duration.create(5, TimeUnit.SECONDS));
}
static
//#context-actorOf
public class FirstActor extends AbstractActor {
final ActorRef child = context().actorOf(Props.create(MyActor.class), "myChild");
final ActorRef child = getContext().actorOf(Props.create(MyActor.class), "myChild");
//#plus-some-behavior
public FirstActor() {
receive(ReceiveBuilder.
matchAny(x -> {
sender().tell(x, self());
}).build()
);
@Override
public Receive createReceive() {
return receiveBuilder()
.matchAny(x -> sender().tell(x, self()))
.build();
}
//#plus-some-behavior
}
//#context-actorOf
static public abstract class SomeActor extends AbstractActor {
//#receive-constructor
public SomeActor() {
receive(ReceiveBuilder.
//#and-some-behavior
match(String.class, s -> System.out.println(s.toLowerCase())).
//#and-some-behavior
build());
}
//#receive-constructor
static public class SomeActor extends AbstractActor {
//#createReceive
@Override
//#receive
public abstract PartialFunction<Object, BoxedUnit> receive();
//#receive
public Receive createReceive() {
return receiveBuilder()
.match(String.class, s -> System.out.println(s.toLowerCase()))
.build();
}
//#createReceive
}
static
//#well-structured
public class WellStructuredActor extends AbstractActor {
public static class Msg1 {}
public static class Msg2 {}
public static class Msg3 {}
@Override
public Receive createReceive() {
return receiveBuilder()
.match(Msg1.class, this::receiveMsg1)
.match(Msg2.class, this::receiveMsg2)
.match(Msg3.class, this::receiveMsg3)
.build();
}
private void receiveMsg1(Msg1 msg) {
// actual work
}
private void receiveMsg2(Msg2 msg) {
// actual work
}
private void receiveMsg3(Msg3 msg) {
// actual work
}
}
//#well-structured
static
//#optimized
public class OptimizedActor extends UntypedAbstractActor {
public static class Msg1 {}
public static class Msg2 {}
public static class Msg3 {}
@Override
public void onReceive(Object msg) throws Exception {
if (msg instanceof Msg1)
receiveMsg1((Msg1) msg);
else if (msg instanceof Msg1)
receiveMsg2((Msg2) msg);
else if (msg instanceof Msg3)
receiveMsg3((Msg3) msg);
else
unhandled(msg);
}
private void receiveMsg1(Msg1 msg) {
// actual work
}
private void receiveMsg2(Msg2 msg) {
// actual work
}
private void receiveMsg3(Msg3 msg) {
// actual work
}
}
//#optimized
static public class ActorWithArgs extends AbstractActor {
private final String args;
ActorWithArgs(String args) {
public ActorWithArgs(String args) {
this.args = args;
receive(ReceiveBuilder.
matchAny(x -> { }).build()
);
}
@Override
public Receive createReceive() {
return receiveBuilder().matchAny(x -> { }).build();
}
}
@ -133,14 +211,18 @@ public class ActorDocTest extends AbstractJavaTest {
}
private final Integer magicNumber;
DemoActor(Integer magicNumber) {
public DemoActor(Integer magicNumber) {
this.magicNumber = magicNumber;
receive(ReceiveBuilder.
match(Integer.class, i -> {
}
@Override
public Receive createReceive() {
return receiveBuilder()
.match(Integer.class, i -> {
sender().tell(i + magicNumber, self());
}).build()
);
})
.build();
}
}
@ -149,11 +231,12 @@ public class ActorDocTest extends AbstractJavaTest {
//#props-factory
public class SomeOtherActor extends AbstractActor {
// Props(new DemoActor(42)) would not be safe
ActorRef demoActor = context().actorOf(DemoActor.props(42), "demo");
ActorRef demoActor = getContext().actorOf(DemoActor.props(42), "demo");
// ...
//#props-factory
public SomeOtherActor() {
receive(emptyBehavior());
@Override
public Receive createReceive() {
return AbstractActor.emptyBehavior();
}
//#props-factory
}
@ -174,26 +257,65 @@ public class ActorDocTest extends AbstractJavaTest {
return from;
}
}
DemoMessagesActor() {
receive(ReceiveBuilder.
match(Greeting.class, g -> {
@Override
public Receive createReceive() {
return receiveBuilder()
.match(Greeting.class, g -> {
log().info("I was greeted by {}", g.getGreeter());
}).build()
);
};
})
.build();
}
}
//#messages-in-companion
public static class LifecycleMethods extends AbstractActor {
@Override
public Receive createReceive() {
return AbstractActor.emptyBehavior();
}
/*
* This section must be kept in sync with the actual Actor trait.
*
* BOYSCOUT RULE: whenever you read this, verify that!
*/
//#lifecycle-callbacks
public void preStart() {
}
public void preRestart(Throwable reason, Optional<Object> message) {
for (ActorRef each : getContext().getChildren()) {
getContext().unwatch(each);
getContext().stop(each);
}
postStop();
}
public void postRestart(Throwable reason) {
preStart();
}
public void postStop() {
}
//#lifecycle-callbacks
}
public static class Hook extends AbstractActor {
ActorRef target = null;
public Hook() {
receive(emptyBehavior());
@Override
public Receive createReceive() {
return AbstractActor.emptyBehavior();
}
//#preStart
@Override
public void preStart() {
target = context().actorOf(Props.create(MyActor.class, "target"));
target = getContext().actorOf(Props.create(MyActor.class, "target"));
}
//#preStart
//#postStop
@ -207,7 +329,7 @@ public class ActorDocTest extends AbstractJavaTest {
//#tell
final Object result = "";
//#forward
target.forward(result, context());
target.forward(result, getContext());
//#forward
target = null;
//#clean-up-some-resources
@ -218,28 +340,30 @@ public class ActorDocTest extends AbstractJavaTest {
public void compileSelections() {
//#selection-local
// will look up this absolute path
context().actorSelection("/user/serviceA/actor");
getContext().actorSelection("/user/serviceA/actor");
// will look up sibling beneath same supervisor
context().actorSelection("../joe");
getContext().actorSelection("../joe");
//#selection-local
//#selection-wildcard
// will look all children to serviceB with names starting with worker
context().actorSelection("/user/serviceB/worker*");
getContext().actorSelection("/user/serviceB/worker*");
// will look up all siblings beneath same supervisor
context().actorSelection("../*");
getContext().actorSelection("../*");
//#selection-wildcard
//#selection-remote
context().actorSelection("akka.tcp://app@otherhost:1234/user/serviceB");
getContext().actorSelection("akka.tcp://app@otherhost:1234/user/serviceB");
//#selection-remote
}
}
public static class ReplyException extends AbstractActor {
public ReplyException() {
receive(ReceiveBuilder.
matchAny(x -> {
@Override
public Receive createReceive() {
return receiveBuilder()
.matchAny(x -> {
//#reply-exception
try {
String result = operation();
@ -249,8 +373,8 @@ public class ActorDocTest extends AbstractJavaTest {
throw e;
}
//#reply-exception
}).build()
);
})
.build();
}
private String operation() {
@ -267,28 +391,31 @@ public class ActorDocTest extends AbstractJavaTest {
public static final Shutdown SHUTDOWN = Shutdown.Shutdown;
private ActorRef worker =
context().watch(context().actorOf(Props.create(Cruncher.class), "worker"));
getContext().watch(getContext().actorOf(Props.create(Cruncher.class), "worker"));
public Manager() {
receive(ReceiveBuilder.
matchEquals("job", s -> {
@Override
public Receive createReceive() {
return receiveBuilder()
.matchEquals("job", s -> {
worker.tell("crunch", self());
}).
matchEquals(SHUTDOWN, x -> {
})
.matchEquals(SHUTDOWN, x -> {
worker.tell(PoisonPill.getInstance(), self());
context().become(shuttingDown);
}).build()
);
getContext().become(shuttingDown());
})
.build();
}
public PartialFunction<Object, BoxedUnit> shuttingDown =
ReceiveBuilder.
matchEquals("job", s -> {
sender().tell("service unavailable, shutting down", self());
}).
match(Terminated.class, t -> t.actor().equals(worker), t -> {
context().stop(self());
}).build();
private AbstractActor.Receive shuttingDown() {
return receiveBuilder()
.matchEquals("job", s ->
sender().tell("service unavailable, shutting down", self())
)
.match(Terminated.class, t -> t.actor().equals(worker), t ->
getContext().stop(self())
)
.build();
}
}
//#gracefulStop-actor
@ -309,27 +436,26 @@ public class ActorDocTest extends AbstractJavaTest {
public static class Cruncher extends AbstractActor {
public Cruncher() {
receive(ReceiveBuilder.
matchEquals("crunch", s -> { }).build()
);
@Override
public Receive createReceive() {
return receiveBuilder().matchEquals("crunch", s -> { }).build();
}
}
static
//#swapper
public class Swapper extends AbstractLoggingActor {
public Swapper() {
receive(ReceiveBuilder.
matchEquals(Swap, s -> {
log().info("Hi");
context().become(ReceiveBuilder.
matchEquals(Swap, x -> {
log().info("Ho");
context().unbecome(); // resets the latest 'become' (just for fun)
}).build(), false); // push on top instead of replace
}).build()
);
@Override
public Receive createReceive() {
return receiveBuilder()
.matchEquals(Swap, s -> {
log().info("Hi");
getContext().become(receiveBuilder().
matchEquals(Swap, x -> {
log().info("Ho");
getContext().unbecome(); // resets the latest 'become' (just for fun)
}).build(), false); // push on top instead of replace
}).build();
}
}
@ -418,29 +544,32 @@ public class ActorDocTest extends AbstractJavaTest {
//#receive-timeout
public class ReceiveTimeoutActor extends AbstractActor {
//#receive-timeout
ActorRef target = context().system().deadLetters();
ActorRef target = getContext().system().deadLetters();
//#receive-timeout
public ReceiveTimeoutActor() {
// To set an initial delay
context().setReceiveTimeout(Duration.create("10 seconds"));
receive(ReceiveBuilder.
matchEquals("Hello", s -> {
getContext().setReceiveTimeout(Duration.create(10, TimeUnit.SECONDS));
}
@Override
public Receive createReceive() {
return receiveBuilder()
.matchEquals("Hello", s -> {
// To set in a response to a message
context().setReceiveTimeout(Duration.create("1 second"));
getContext().setReceiveTimeout(Duration.create(1, TimeUnit.SECONDS));
//#receive-timeout
target = sender();
target.tell("Hello world", self());
//#receive-timeout
}).
match(ReceiveTimeout.class, r -> {
})
.match(ReceiveTimeout.class, r -> {
// To turn it off
context().setReceiveTimeout(Duration.Undefined());
getContext().setReceiveTimeout(Duration.Undefined());
//#receive-timeout
target.tell("timeout", self());
//#receive-timeout
}).build()
);
})
.build();
}
}
//#receive-timeout
@ -460,35 +589,40 @@ public class ActorDocTest extends AbstractJavaTest {
static
//#hot-swap-actor
public class HotSwapActor extends AbstractActor {
private PartialFunction<Object, BoxedUnit> angry;
private PartialFunction<Object, BoxedUnit> happy;
private AbstractActor.Receive angry;
private AbstractActor.Receive happy;
public HotSwapActor() {
angry =
ReceiveBuilder.
matchEquals("foo", s -> {
receiveBuilder()
.matchEquals("foo", s -> {
sender().tell("I am already angry?", self());
}).
matchEquals("bar", s -> {
context().become(happy);
}).build();
})
.matchEquals("bar", s -> {
getContext().become(happy);
})
.build();
happy = ReceiveBuilder.
matchEquals("bar", s -> {
happy = receiveBuilder()
.matchEquals("bar", s -> {
sender().tell("I am already happy :-)", self());
}).
matchEquals("foo", s -> {
context().become(angry);
}).build();
receive(ReceiveBuilder.
matchEquals("foo", s -> {
context().become(angry);
}).
matchEquals("bar", s -> {
context().become(happy);
}).build()
);
})
.matchEquals("foo", s -> {
getContext().become(angry);
})
.build();
}
@Override
public Receive createReceive() {
return receiveBuilder()
.matchEquals("foo", s ->
getContext().become(angry)
)
.matchEquals("bar", s ->
getContext().become(happy)
)
.build();
}
}
//#hot-swap-actor
@ -516,19 +650,21 @@ public class ActorDocTest extends AbstractJavaTest {
static
//#stash
public class ActorWithProtocol extends AbstractActorWithStash {
public ActorWithProtocol() {
receive(ReceiveBuilder.
matchEquals("open", s -> {
context().become(ReceiveBuilder.
matchEquals("write", ws -> { /* do writing */ }).
matchEquals("close", cs -> {
@Override
public Receive createReceive() {
return receiveBuilder()
.matchEquals("open", s -> {
getContext().become(receiveBuilder()
.matchEquals("write", ws -> { /* do writing */ })
.matchEquals("close", cs -> {
unstashAll();
context().unbecome();
}).
matchAny(msg -> stash()).build(), false);
}).
matchAny(msg -> stash()).build()
);
getContext().unbecome();
})
.matchAny(msg -> stash())
.build(), false);
})
.matchAny(msg -> stash())
.build();
}
}
//#stash
@ -541,21 +677,24 @@ public class ActorDocTest extends AbstractJavaTest {
static
//#watch
public class WatchActor extends AbstractActor {
private final ActorRef child = context().actorOf(Props.empty(), "target");
private final ActorRef child = getContext().actorOf(Props.empty(), "target");
private ActorRef lastSender = system.deadLetters();
public WatchActor() {
context().watch(child); // <-- this is the only call needed for registration
receive(ReceiveBuilder.
matchEquals("kill", s -> {
context().stop(child);
getContext().watch(child); // <-- this is the only call needed for registration
}
@Override
public Receive createReceive() {
return receiveBuilder()
.matchEquals("kill", s -> {
getContext().stop(child);
lastSender = sender();
}).
match(Terminated.class, t -> t.actor().equals(child), t -> {
})
.match(Terminated.class, t -> t.actor().equals(child), t -> {
lastSender.tell("finished", self());
}).build()
);
})
.build();
}
}
//#watch
@ -578,26 +717,30 @@ public class ActorDocTest extends AbstractJavaTest {
final Integer identifyId = 1;
public Follower(){
ActorSelection selection = context().actorSelection("/user/another");
ActorSelection selection = getContext().actorSelection("/user/another");
selection.tell(new Identify(identifyId), self());
receive(ReceiveBuilder.
match(ActorIdentity.class, id -> id.getRef() != null, id -> {
ActorRef ref = id.getRef();
context().watch(ref);
context().become(active(ref));
}).
match(ActorIdentity.class, id -> id.getRef() == null, id -> {
context().stop(self());
}).build()
);
}
final PartialFunction<Object, BoxedUnit> active(final ActorRef another) {
return ReceiveBuilder.
match(Terminated.class, t -> t.actor().equals(another), t -> {
context().stop(self());
}).build();
@Override
public Receive createReceive() {
return receiveBuilder()
.match(ActorIdentity.class, id -> id.getActorRef().isPresent(), id -> {
ActorRef ref = id.getActorRef().get();
getContext().watch(ref);
getContext().become(active(ref));
})
.match(ActorIdentity.class, id -> !id.getActorRef().isPresent(), id -> {
getContext().stop(self());
})
.build();
}
final AbstractActor.Receive active(final ActorRef another) {
return receiveBuilder()
.match(Terminated.class, t -> t.actor().equals(another), t ->
getContext().stop(self())
)
.build();
}
}
//#identify
@ -615,73 +758,70 @@ public class ActorDocTest extends AbstractJavaTest {
}
};
}
public static class NoReceiveActor extends AbstractActor {
}
@Test
public void noReceiveActor() {
EventFilter ex1 = new ErrorFilter(ActorInitializationException.class);
EventFilter[] ignoreExceptions = { ex1 };
try {
system.eventStream().publish(new TestEvent.Mute(immutableSeq(ignoreExceptions)));
new JavaTestKit(system) {{
final ActorRef victim = new EventFilter<ActorRef>(ActorInitializationException.class) {
protected ActorRef run() {
return system.actorOf(Props.create(NoReceiveActor.class), "victim");
}
}.message("Actor behavior has not been set with receive(...)").occurrences(1).exec();
public void usePatternsAskPipe() {
new JavaTestKit(system) {
{
ActorRef actorA = system.actorOf(TestActors.echoActorProps());
ActorRef actorB = system.actorOf(TestActors.echoActorProps());
ActorRef actorC = getRef();
assertEquals(true, victim.isTerminated());
}};
} finally {
system.eventStream().publish(new TestEvent.UnMute(immutableSeq(ignoreExceptions)));
}
//#ask-pipe
final Timeout t = new Timeout(Duration.create(5, TimeUnit.SECONDS));
final ArrayList<Future<Object>> futures = new ArrayList<Future<Object>>();
futures.add(ask(actorA, "request", 1000)); // using 1000ms timeout
futures.add(ask(actorB, "another request", t)); // using timeout from
// above
final Future<Iterable<Object>> aggregate = Futures.sequence(futures,
system.dispatcher());
final Future<Result> transformed = aggregate.map(
new Mapper<Iterable<Object>, Result>() {
public Result apply(Iterable<Object> coll) {
final Iterator<Object> it = coll.iterator();
final String x = (String) it.next();
final String s = (String) it.next();
return new Result(x, s);
}
}, system.dispatcher());
pipe(transformed, system.dispatcher()).to(actorC);
//#ask-pipe
expectMsgEquals(new Result("request", "another request"));
}
};
}
public static class MultipleReceiveActor extends AbstractActor {
public MultipleReceiveActor() {
receive(ReceiveBuilder.
match(String.class, s1 -> s1.toLowerCase().equals("become"), s1 -> {
sender().tell(s1.toUpperCase(), self());
receive(ReceiveBuilder.
match(String.class, s2 -> {
sender().tell(s2.toLowerCase(), self());
}).build()
);
}).
match(String.class, s1 -> {
sender().tell(s1.toUpperCase(), self());
}).build()
);
}
}
@Test
public void multipleReceiveActor() {
EventFilter ex1 = new ErrorFilter(IllegalActorStateException.class);
EventFilter[] ignoreExceptions = { ex1 };
try {
system.eventStream().publish(new TestEvent.Mute(immutableSeq(ignoreExceptions)));
new JavaTestKit(system) {{
new EventFilter<Boolean>(IllegalActorStateException.class) {
protected Boolean run() {
ActorRef victim = system.actorOf(Props.create(MultipleReceiveActor.class), "victim2");
victim.tell("Foo", getRef());
expectMsgEquals("FOO");
victim.tell("bEcoMe", getRef());
expectMsgEquals("BECOME");
victim.tell("Foo", getRef());
// if it's upper case, then the actor was restarted
expectMsgEquals("FOO");
return true;
}
}.message("Actor behavior has already been set with receive(...), " +
"use context().become(...) to change it later").occurrences(1).exec();
}};
} finally {
system.eventStream().publish(new TestEvent.UnMute(immutableSeq(ignoreExceptions)));
}
public void useKill() {
new JavaTestKit(system) {
{
ActorRef victim = system.actorOf(TestActors.echoActorProps());
watch(victim);
//#kill
victim.tell(akka.actor.Kill.getInstance(), ActorRef.noSender());
//#kill
expectTerminated(Duration.create(3, TimeUnit.SECONDS), victim);
}
};
}
@Test
public void usePoisonPill() {
new JavaTestKit(system) {
{
ActorRef victim = system.actorOf(TestActors.echoActorProps());
watch(victim);
//#poison-pill
victim.tell(akka.actor.PoisonPill.getInstance(), ActorRef.noSender());
//#poison-pill
expectTerminated(Duration.create(3, TimeUnit.SECONDS), victim);
}
};
}
@Test
@ -691,7 +831,7 @@ public class ActorDocTest extends AbstractJavaTest {
CoordinatedShutdown.get(system).addTask(
CoordinatedShutdown.PhaseBeforeServiceUnbind(), "someTaskName",
() -> {
return ask(someActor, "stop", new Timeout(5, TimeUnit.SECONDS))
return akka.pattern.PatternsCS.ask(someActor, "stop", new Timeout(5, TimeUnit.SECONDS))
.thenApply(reply -> Done.getInstance());
});
//#coordinated-shutdown-addTask

View file

@ -0,0 +1,105 @@
/**
* Copyright (C) 2009-2017 Lightbend Inc. <http://www.lightbend.com>
*/
package docs.actorlambda;
import docs.AbstractJavaTest;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import scala.concurrent.Await;
import scala.concurrent.duration.Duration;
import akka.actor.AbstractActor;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.actor.Props;
import akka.testkit.JavaTestKit;
//#import
import akka.actor.Actor;
import akka.actor.IndirectActorProducer;
//#import
public class DependencyInjectionDocTest extends AbstractJavaTest {
public static class TheActor extends AbstractActor {
final String s;
public TheActor(String s) {
this.s = s;
}
@Override
public Receive createReceive() {
return receiveBuilder()
.match(String.class, msg -> {
sender().tell(s, self());
})
.build();
}
}
static ActorSystem system = null;
@BeforeClass
public static void beforeClass() {
system = ActorSystem.create("DependencyInjectionDocTest");
}
@AfterClass
public static void afterClass() throws Exception {
Await.ready(system.terminate(), Duration.create("5 seconds"));
}
//this is just to make the test below a tiny fraction nicer
private ActorSystem getContext() {
return system;
}
static
//#creating-indirectly
class DependencyInjector implements IndirectActorProducer {
final Object applicationContext;
final String beanName;
public DependencyInjector(Object applicationContext, String beanName) {
this.applicationContext = applicationContext;
this.beanName = beanName;
}
@Override
public Class<? extends Actor> actorClass() {
return TheActor.class;
}
@Override
public TheActor produce() {
TheActor result;
//#obtain-fresh-Actor-instance-from-DI-framework
result = new TheActor((String) applicationContext);
//#obtain-fresh-Actor-instance-from-DI-framework
return result;
}
}
//#creating-indirectly
@Test
public void indirectActorOf() {
final String applicationContext = "...";
//#creating-indirectly
final ActorRef myActor = getContext().actorOf(
Props.create(DependencyInjector.class, applicationContext, "TheActor"),
"TheActor");
//#creating-indirectly
new JavaTestKit(system) {
{
myActor.tell("hello", getRef());
expectMsgEquals("...");
}
};
}
}

View file

@ -3,37 +3,41 @@
*/
package docs.actorlambda;
//#testkit
import akka.actor.*;
import com.typesafe.config.Config;
import com.typesafe.config.ConfigFactory;
import docs.AbstractJavaTest;
import java.util.Optional;
import static akka.pattern.Patterns.ask;
//#testkit
import akka.testkit.JavaTestKit;
import akka.testkit.TestProbe;
import akka.testkit.ErrorFilter;
import akka.testkit.EventFilter;
import akka.testkit.TestEvent;
import scala.concurrent.duration.Duration;
import static java.util.concurrent.TimeUnit.SECONDS;
import static akka.japi.Util.immutableSeq;
import scala.concurrent.Await;
//#testkit
//#supervisor
import akka.japi.pf.DeciderBuilder;
import static akka.actor.SupervisorStrategy.resume;
import static akka.actor.SupervisorStrategy.restart;
import static akka.actor.SupervisorStrategy.stop;
import static akka.actor.SupervisorStrategy.escalate;
import akka.japi.pf.DeciderBuilder;
import akka.japi.pf.ReceiveBuilder;
import com.typesafe.config.Config;
import com.typesafe.config.ConfigFactory;
import docs.AbstractJavaTest;
import scala.PartialFunction;
import scala.concurrent.Await;
import static akka.pattern.Patterns.ask;
import scala.concurrent.duration.Duration;
import akka.testkit.TestProbe;
//#testkit
import akka.testkit.ErrorFilter;
import akka.testkit.EventFilter;
import akka.testkit.TestEvent;
import akka.testkit.JavaTestKit;
import static java.util.concurrent.TimeUnit.SECONDS;
import static akka.japi.Util.immutableSeq;
import scala.Option;
//#supervisor
import org.junit.Test;
import org.junit.BeforeClass;
import org.junit.AfterClass;
import scala.runtime.BoxedUnit;
//#testkit
public class FaultHandlingTest extends AbstractJavaTest {
@ -65,12 +69,13 @@ public class FaultHandlingTest extends AbstractJavaTest {
//#strategy
public Supervisor() {
receive(ReceiveBuilder.
match(Props.class, props -> {
sender().tell(context().actorOf(props), self());
}).build()
);
@Override
public Receive createReceive() {
return receiveBuilder()
.match(Props.class, props -> {
sender().tell(getContext().actorOf(props), self());
})
.build();
}
}
@ -95,16 +100,17 @@ public class FaultHandlingTest extends AbstractJavaTest {
//#strategy2
public Supervisor2() {
receive(ReceiveBuilder.
match(Props.class, props -> {
sender().tell(context().actorOf(props), self());
}).build()
);
@Override
public Receive createReceive() {
return receiveBuilder()
.match(Props.class, props -> {
sender().tell(getContext().actorOf(props), self());
})
.build();
}
@Override
public void preRestart(Throwable cause, Option<Object> msg) {
public void preRestart(Throwable cause, Optional<Object> msg) {
// do not kill all children, which is the default here
}
}
@ -116,12 +122,13 @@ public class FaultHandlingTest extends AbstractJavaTest {
public class Child extends AbstractActor {
int state = 0;
public Child() {
receive(ReceiveBuilder.
match(Exception.class, exception -> { throw exception; }).
match(Integer.class, i -> state = i).
matchEquals("get", s -> sender().tell(state, self())).build()
);
@Override
public Receive createReceive() {
return receiveBuilder()
.match(Exception.class, exception -> { throw exception; })
.match(Integer.class, i -> state = i)
.matchEquals("get", s -> sender().tell(state, self()))
.build();
}
}

View file

@ -15,10 +15,12 @@ import akka.japi.pf.UnitPFBuilder;
//#actor
public class GraduallyBuiltActor extends AbstractActor {
private final LoggingAdapter log = Logging.getLogger(context().system(), this);
private final LoggingAdapter log = Logging.getLogger(getContext().system(), this);
public GraduallyBuiltActor() {
UnitPFBuilder<Object> builder = ReceiveBuilder.create();
@Override
public Receive createReceive() {
ReceiveBuilder builder = ReceiveBuilder.create();
builder.match(String.class, s -> {
log.info("Received String message: {}", s);
//#actor
@ -27,9 +29,12 @@ public class GraduallyBuiltActor extends AbstractActor {
//#reply
//#actor
});
// do some other stuff in between
builder.matchAny(o -> log.info("received unknown message"));
receive(builder.build());
return builder.build();
}
}
//#actor

View file

@ -8,7 +8,6 @@ import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.actor.Props;
import akka.japi.pf.FI;
import akka.japi.pf.ReceiveBuilder;
import akka.testkit.JavaTestKit;
import docs.AbstractJavaTest;
import org.junit.AfterClass;
@ -16,7 +15,7 @@ import org.junit.BeforeClass;
import org.junit.Test;
import scala.concurrent.Await;
import scala.concurrent.duration.Duration;
import java.util.Optional;
import java.util.concurrent.TimeUnit;
public class InitializationDocTest extends AbstractJavaTest {
@ -32,24 +31,57 @@ public class InitializationDocTest extends AbstractJavaTest {
public static void afterClass() throws Exception {
Await.ready(system.terminate(), Duration.create("5 seconds"));
}
static public class PreStartInitExample extends AbstractActor {
@Override
public Receive createReceive() {
return AbstractActor.emptyBehavior();
}
//#preStartInit
@Override
public void preStart() {
// Initialize children here
}
// Overriding postRestart to disable the call to preStart()
// after restarts
@Override
public void postRestart(Throwable reason) {
}
// The default implementation of preRestart() stops all the children
// of the actor. To opt-out from stopping the children, we
// have to override preRestart()
@Override
public void preRestart(Throwable reason, Optional<Object> message)
throws Exception {
// Keep the call to postStop(), but no stopping of children
postStop();
}
//#preStartInit
}
public static class MessageInitExample extends AbstractActor {
private String initializeMe = null;
public MessageInitExample() {
//#messageInit
receive(ReceiveBuilder.
matchEquals("init", m1 -> {
initializeMe = "Up and running";
context().become(ReceiveBuilder.
matchEquals("U OK?", m2 -> {
sender().tell(initializeMe, self());
}).build());
}).build()
//#messageInit
);
//#messageInit
@Override
public Receive createReceive() {
return receiveBuilder()
.matchEquals("init", m1 -> {
initializeMe = "Up and running";
getContext().become(receiveBuilder()
.matchEquals("U OK?", m2 -> {
sender().tell(initializeMe, self());
})
.build());
})
.build();
}
//#messageInit
}
public class GenericMessage<T> {
@ -61,24 +93,27 @@ public class InitializationDocTest extends AbstractJavaTest {
}
public static class GenericActor extends AbstractActor {
public GenericActor() {
receive(ReceiveBuilder.match(GenericMessage.class, (GenericMessage<String> msg) -> {
GenericMessage<String> message = msg;
sender().tell(message.value.toUpperCase(), self());
}).build());
@Override
public Receive createReceive() {
return receiveBuilder()
.matchUnchecked(GenericMessage.class, (GenericMessage<String> msg) -> {
GenericMessage<String> message = msg;
sender().tell(message.value.toUpperCase(), self());
})
.build();
}
}
static class GenericActorWithPredicate extends AbstractActor {
public GenericActorWithPredicate() {
@Override
public Receive createReceive() {
FI.TypedPredicate<GenericMessage<String>> typedPredicate = s -> !s.value.isEmpty();
receive(ReceiveBuilder.match(GenericMessage.class, typedPredicate, (GenericMessage<String> msg) -> {
sender().tell(msg.value.toUpperCase(), self());
}).build());
return receiveBuilder()
.matchUnchecked(GenericMessage.class, typedPredicate, (GenericMessage<String> msg) -> {
sender().tell(msg.value.toUpperCase(), self());
})
.build();
}
}

View file

@ -8,26 +8,26 @@ package docs.actorlambda;
import akka.actor.AbstractActor;
import akka.event.Logging;
import akka.event.LoggingAdapter;
import akka.japi.pf.ReceiveBuilder;
//#imports
//#my-actor
public class MyActor extends AbstractActor {
private final LoggingAdapter log = Logging.getLogger(context().system(), this);
private final LoggingAdapter log = Logging.getLogger(getContext().system(), this);
public MyActor() {
receive(ReceiveBuilder.
match(String.class, s -> {
@Override
public Receive createReceive() {
return receiveBuilder()
.match(String.class, s -> {
log.info("Received String message: {}", s);
//#my-actor
//#reply
sender().tell(s, self());
//#reply
//#my-actor
}).
matchAny(o -> log.info("received unknown message")).build()
);
})
.matchAny(o -> log.info("received unknown message"))
.build();
}
}
//#my-actor

View file

@ -0,0 +1,14 @@
/**
* Copyright (C) 2009-2017 Lightbend Inc. <http://www.lightbend.com>
*/
package docs.actorlambda;
//#my-bounded-untyped-actor
import akka.dispatch.BoundedMessageQueueSemantics;
import akka.dispatch.RequiresMessageQueue;
public class MyBoundedActor extends MyActor
implements RequiresMessageQueue<BoundedMessageQueueSemantics> {
}
//#my-bounded-untyped-actor

View file

@ -0,0 +1,29 @@
/**
* Copyright (C) 2009-2017 Lightbend Inc. <http://www.lightbend.com>
*/
package docs.actorlambda;
//#my-stopping-actor
import akka.actor.ActorRef;
import akka.actor.AbstractActor;
public class MyStoppingActor extends AbstractActor {
ActorRef child = null;
// ... creation of child ...
@Override
public Receive createReceive() {
return receiveBuilder()
.matchEquals("interrupt-child", m ->
getContext().stop(child)
)
.matchEquals("done", m ->
getContext().stop(self())
)
.build();
}
}
//#my-stopping-actor

View file

@ -7,30 +7,30 @@ package docs.actorlambda;
//#sample-actor
import akka.actor.AbstractActor;
import akka.japi.pf.ReceiveBuilder;
import scala.PartialFunction;
import scala.runtime.BoxedUnit;
public class SampleActor extends AbstractActor {
private PartialFunction<Object, BoxedUnit> guarded = ReceiveBuilder.
match(String.class, s -> s.contains("guard"), s -> {
private Receive guarded = receiveBuilder()
.match(String.class, s -> s.contains("guard"), s -> {
sender().tell("contains(guard): " + s, self());
context().unbecome();
}).build();
getContext().unbecome();
})
.build();
public SampleActor() {
receive(ReceiveBuilder.
match(Double.class, d -> {
@Override
public Receive createReceive() {
return receiveBuilder()
.match(Double.class, d -> {
sender().tell(d.isNaN() ? 0 : d, self());
}).
match(Integer.class, i -> {
})
.match(Integer.class, i -> {
sender().tell(i * 10, self());
}).
match(String.class, s -> s.startsWith("guard"), s -> {
})
.match(String.class, s -> s.startsWith("guard"), s -> {
sender().tell("startsWith(guard): " + s.toUpperCase(), self());
context().become(guarded, false);
}).build()
);
getContext().become(guarded, false);
})
.build();
}
}
//#sample-actor

View file

@ -174,7 +174,7 @@ public class FSMDocTest extends AbstractJavaTest {
expectMsgEquals(Active);
expectMsgEquals(Data.Foo);
String msg = expectMsgClass(String.class);
assertTrue(msg.startsWith("LogEntry(SomeState,Foo,Actor[akka://FSMDocTest/system/"));
assertThat(msg, CoreMatchers.startsWith("LogEntry(SomeState,Foo,Actor[akka://FSMDocTest/system/"));
}};
}
}

View file

@ -67,24 +67,24 @@ public class FaultHandlingDocSample {
public void preStart() {
// If we don't get any progress within 15 seconds then the service
// is unavailable
context().setReceiveTimeout(Duration.create("15 seconds"));
getContext().setReceiveTimeout(Duration.create("15 seconds"));
}
public Listener() {
receive(LoggingReceive.create(ReceiveBuilder.
@Override
public Receive createReceive() {
return LoggingReceive.create(receiveBuilder().
match(Progress.class, progress -> {
log().info("Current progress: {} %", progress.percent);
if (progress.percent >= 100.0) {
log().info("That's all, shutting down");
context().system().terminate();
getContext().system().terminate();
}
}).
matchEquals(ReceiveTimeout.getInstance(), x -> {
// No progress within 15 seconds, ServiceUnavailable
log().error("Shutting down due to unavailable service");
context().system().terminate();
}).build(), context()
));
getContext().system().terminate();
}).build(), getContext());
}
}
@ -119,7 +119,7 @@ public class FaultHandlingDocSample {
// The sender of the initial Start message will continuously be notified
// about progress
ActorRef progressListener;
final ActorRef counterService = context().actorOf(
final ActorRef counterService = getContext().actorOf(
Props.create(CounterService.class), "counter");
final int totalCount = 51;
@ -134,13 +134,14 @@ public class FaultHandlingDocSample {
return strategy;
}
public Worker() {
receive(LoggingReceive.create(ReceiveBuilder.
@Override
public Receive createReceive() {
return LoggingReceive.create(receiveBuilder().
matchEquals(Start, x -> progressListener == null, x -> {
progressListener = sender();
context().system().scheduler().schedule(
getContext().system().scheduler().schedule(
Duration.Zero(), Duration.create(1, "second"), self(), Do,
context().dispatcher(), null
getContext().dispatcher(), null
);
}).
matchEquals(Do, x -> {
@ -154,10 +155,9 @@ public class FaultHandlingDocSample {
public Progress apply(CurrentCount c) {
return new Progress(100.0 * c.count / totalCount);
}
}, context().dispatcher()), context().dispatcher())
}, getContext().dispatcher()), getContext().dispatcher())
.to(progressListener);
}).build(), context())
);
}).build(), getContext());
}
}
@ -254,7 +254,7 @@ public class FaultHandlingDocSample {
* when it has been terminated.
*/
void initStorage() {
storage = context().watch(context().actorOf(
storage = getContext().watch(getContext().actorOf(
Props.create(Storage.class), "storage"));
// Tell the counter, if any, to use the new storage
if (counter != null)
@ -263,12 +263,13 @@ public class FaultHandlingDocSample {
storage.tell(new Get(key), self());
}
public CounterService() {
receive(LoggingReceive.create(ReceiveBuilder.
@Override
public Receive createReceive() {
return LoggingReceive.create(receiveBuilder().
match(Entry.class, entry -> entry.key.equals(key) && counter == null, entry -> {
// Reply from Storage of the initial value, now we can create the Counter
final long value = entry.value;
counter = context().actorOf(Props.create(Counter.class, key, value));
counter = getContext().actorOf(Props.create(Counter.class, key, value));
// Tell the counter to use current storage
counter.tell(new UseStorage(storage), self());
// and send the buffered backlog to the counter
@ -290,15 +291,14 @@ public class FaultHandlingDocSample {
// Tell the counter that there is no storage for the moment
counter.tell(new UseStorage(null), self());
// Try to re-establish storage after while
context().system().scheduler().scheduleOnce(
getContext().system().scheduler().scheduleOnce(
Duration.create(10, "seconds"), self(), Reconnect,
context().dispatcher(), null);
getContext().dispatcher(), null);
}).
matchEquals(Reconnect, o -> {
// Re-establish storage after the scheduled delay
initStorage();
}).build(), context())
);
}).build(), getContext());
}
void forwardOrPlaceInBacklog(Object msg) {
@ -311,7 +311,7 @@ public class FaultHandlingDocSample {
" lack of initial value");
backlog.add(new SenderMsgPair(sender(), msg));
} else {
counter.forward(msg, context());
counter.forward(msg, getContext());
}
}
}
@ -345,8 +345,11 @@ public class FaultHandlingDocSample {
public Counter(String key, long initialValue) {
this.key = key;
this.count = initialValue;
receive(LoggingReceive.create(ReceiveBuilder.
}
@Override
public Receive createReceive() {
return LoggingReceive.create(receiveBuilder().
match(UseStorage.class, useStorage -> {
storage = useStorage.storage;
storeCount();
@ -357,8 +360,7 @@ public class FaultHandlingDocSample {
}).
matchEquals(GetCurrentCount, gcc -> {
sender().tell(new CurrentCount(key, count), self());
}).build(), context())
);
}).build(), getContext());
}
void storeCount() {
@ -430,8 +432,9 @@ public class FaultHandlingDocSample {
final DummyDB db = DummyDB.instance;
public Storage() {
receive(LoggingReceive.create(ReceiveBuilder.
@Override
public Receive createReceive() {
return LoggingReceive.create(receiveBuilder().
match(Store.class, store -> {
db.save(store.entry.key, store.entry.value);
}).
@ -439,8 +442,7 @@ public class FaultHandlingDocSample {
Long value = db.load(get.key);
sender().tell(new Entry(get.key, value == null ?
Long.valueOf(0L) : value), self());
}).build(), context())
);
}).build(), getContext());
}
}