Merge branch 'master' into wip-sync-artery-patriknw

This commit is contained in:
Patrik Nordwall 2016-06-03 11:09:17 +02:00
commit 839ec5f167
757 changed files with 9166 additions and 6642 deletions

View file

@ -5,6 +5,10 @@
package akka.actor; package akka.actor;
import static org.junit.Assert.*; import static org.junit.Assert.*;
import static java.util.stream.Collectors.toCollection;
import java.util.ArrayList;
import java.util.stream.IntStream;
import akka.testkit.TestActors; import akka.testkit.TestActors;
import org.junit.Test; import org.junit.Test;
@ -144,6 +148,30 @@ public class ActorCreationTest extends JUnitSuite {
} }
} }
public static class Issue20537Reproducer extends UntypedActor {
static final class ReproducerCreator implements Creator<Issue20537Reproducer> {
final boolean create;
private ReproducerCreator(boolean create) {
this.create = create;
}
@Override
public Issue20537Reproducer create() throws Exception {
return new Issue20537Reproducer(create);
}
}
public Issue20537Reproducer(boolean create) {
}
@Override
public void onReceive(Object message) throws Exception {
}
}
@Test @Test
public void testWrongAnonymousInPlaceCreator() { public void testWrongAnonymousInPlaceCreator() {
try { try {
@ -287,5 +315,18 @@ public class ActorCreationTest extends JUnitSuite {
assertEquals(UntypedTestActor.class, p.actorClass()); assertEquals(UntypedTestActor.class, p.actorClass());
} }
@Test
public void testIssue20537Reproducer() {
final Issue20537Reproducer.ReproducerCreator creator = new Issue20537Reproducer.ReproducerCreator(false);
final Props p = Props.create(creator);
assertEquals(Issue20537Reproducer.class, p.actorClass());
ArrayList<Props> pList = IntStream.range(0, 4).mapToObj(i -> Props.create(creator))
.collect(toCollection(ArrayList::new));
for (Props each : pList) {
assertEquals(Issue20537Reproducer.class, each.actorClass());
}
}
} }

View file

@ -63,7 +63,7 @@ public class JavaFutureTests extends JUnitSuite {
}, system.dispatcher()); }, system.dispatcher());
cf.success("foo"); cf.success("foo");
assertTrue(latch.await(5000, TimeUnit.MILLISECONDS)); assertTrue(latch.await(5, TimeUnit.SECONDS));
assertEquals(Await.result(f, timeout), "foo"); assertEquals(Await.result(f, timeout), "foo");
} }
@ -81,7 +81,7 @@ public class JavaFutureTests extends JUnitSuite {
Throwable exception = new NullPointerException(); Throwable exception = new NullPointerException();
cf.failure(exception); cf.failure(exception);
assertTrue(latch.await(5000, TimeUnit.MILLISECONDS)); assertTrue(latch.await(5, TimeUnit.SECONDS));
assertEquals(f.value().get().failed().get(), exception); assertEquals(f.value().get().failed().get(), exception);
} }
@ -97,7 +97,7 @@ public class JavaFutureTests extends JUnitSuite {
}, system.dispatcher()); }, system.dispatcher());
cf.success("foo"); cf.success("foo");
assertTrue(latch.await(5000, TimeUnit.MILLISECONDS)); assertTrue(latch.await(5, TimeUnit.SECONDS));
assertEquals(Await.result(f, timeout), "foo"); assertEquals(Await.result(f, timeout), "foo");
} }
@ -113,7 +113,7 @@ public class JavaFutureTests extends JUnitSuite {
},system.dispatcher()); },system.dispatcher());
cf.success("foo"); cf.success("foo");
assertTrue(latch.await(5000, TimeUnit.MILLISECONDS)); assertTrue(latch.await(5, TimeUnit.SECONDS));
assertEquals(Await.result(f, timeout), "foo"); assertEquals(Await.result(f, timeout), "foo");
} }
@ -135,7 +135,7 @@ public class JavaFutureTests extends JUnitSuite {
assertEquals(Await.result(f, timeout), "1000"); assertEquals(Await.result(f, timeout), "1000");
assertEquals(Await.result(r, timeout).intValue(), 1000); assertEquals(Await.result(r, timeout).intValue(), 1000);
assertTrue(latch.await(5000, TimeUnit.MILLISECONDS)); assertTrue(latch.await(5, TimeUnit.SECONDS));
} }
@Test @Test
@ -151,7 +151,7 @@ public class JavaFutureTests extends JUnitSuite {
}), system.dispatcher()); }), system.dispatcher());
cf.success("foo"); cf.success("foo");
assertTrue(latch.await(5000, TimeUnit.MILLISECONDS)); assertTrue(latch.await(5, TimeUnit.SECONDS));
assertEquals(Await.result(f, timeout), "foo"); assertEquals(Await.result(f, timeout), "foo");
assertEquals(Await.result(r, timeout), "foo"); assertEquals(Await.result(r, timeout), "foo");
} }

View file

@ -3,6 +3,7 @@ package akka.event;
import akka.actor.ActorRef; import akka.actor.ActorRef;
import akka.actor.ActorSystem; import akka.actor.ActorSystem;
import akka.actor.Props; import akka.actor.Props;
import akka.testkit.AkkaJUnitActorSystemResource;
import akka.testkit.JavaTestKit; import akka.testkit.JavaTestKit;
import akka.event.Logging.Error; import akka.event.Logging.Error;
import akka.event.ActorWithMDC.Log; import akka.event.ActorWithMDC.Log;
@ -10,6 +11,8 @@ import static akka.event.Logging.*;
import com.typesafe.config.Config; import com.typesafe.config.Config;
import com.typesafe.config.ConfigFactory; import com.typesafe.config.ConfigFactory;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;
import scala.concurrent.duration.Duration; import scala.concurrent.duration.Duration;
@ -30,9 +33,19 @@ public class LoggingAdapterTest extends JUnitSuite {
"akka.actor.serialize-messages = off" "akka.actor.serialize-messages = off"
); );
@Rule
public AkkaJUnitActorSystemResource actorSystemResource =
new AkkaJUnitActorSystemResource("LoggingAdapterTest", config);
private ActorSystem system = null;
@Before
public void beforeEach() {
system = actorSystemResource.getSystem();
}
@Test @Test
public void mustFormatMessage() { public void mustFormatMessage() {
final ActorSystem system = ActorSystem.create("test-system", config);
final LoggingAdapter log = Logging.getLogger(system, this); final LoggingAdapter log = Logging.getLogger(system, this);
new LogJavaTestKit(system) {{ new LogJavaTestKit(system) {{
system.eventStream().subscribe(getRef(), LogEvent.class); system.eventStream().subscribe(getRef(), LogEvent.class);
@ -66,7 +79,6 @@ public class LoggingAdapterTest extends JUnitSuite {
@Test @Test
public void mustCallMdcForEveryLog() throws Exception { public void mustCallMdcForEveryLog() throws Exception {
final ActorSystem system = ActorSystem.create("test-system", config);
new LogJavaTestKit(system) {{ new LogJavaTestKit(system) {{
system.eventStream().subscribe(getRef(), LogEvent.class); system.eventStream().subscribe(getRef(), LogEvent.class);
ActorRef ref = system.actorOf(Props.create(ActorWithMDC.class)); ActorRef ref = system.actorOf(Props.create(ActorWithMDC.class));
@ -86,7 +98,6 @@ public class LoggingAdapterTest extends JUnitSuite {
@Test @Test
public void mustSupportMdcNull() throws Exception { public void mustSupportMdcNull() throws Exception {
final ActorSystem system = ActorSystem.create("test-system", config);
new LogJavaTestKit(system) {{ new LogJavaTestKit(system) {{
system.eventStream().subscribe(getRef(), LogEvent.class); system.eventStream().subscribe(getRef(), LogEvent.class);
ActorRef ref = system.actorOf(Props.create(ActorWithMDC.class)); ActorRef ref = system.actorOf(Props.create(ActorWithMDC.class));
@ -131,6 +142,7 @@ public class LoggingAdapterTest extends JUnitSuite {
void expectLog(final Object level, final String message, final Throwable cause, final String mdc) { void expectLog(final Object level, final String message, final Throwable cause, final String mdc) {
new ExpectMsg<Void>(Duration.create(3, TimeUnit.SECONDS), "LogEvent") { new ExpectMsg<Void>(Duration.create(3, TimeUnit.SECONDS), "LogEvent") {
@Override
protected Void match(Object event) { protected Void match(Object event) {
LogEvent log = (LogEvent) event; LogEvent log = (LogEvent) event;
assertEquals(message, log.message()); assertEquals(message, log.message());

View file

@ -101,7 +101,8 @@ class ActorLifeCycleSpec extends AkkaSpec("akka.actor.serialize-messages=off") w
"not invoke preRestart and postRestart when never restarted using OneForOneStrategy" in { "not invoke preRestart and postRestart when never restarted using OneForOneStrategy" in {
val id = newUuid().toString val id = newUuid().toString
val supervisor = system.actorOf(Props(classOf[Supervisor], val supervisor = system.actorOf(Props(
classOf[Supervisor],
OneForOneStrategy(maxNrOfRetries = 3)(List(classOf[Exception])))) OneForOneStrategy(maxNrOfRetries = 3)(List(classOf[Exception]))))
val gen = new AtomicInteger(0) val gen = new AtomicInteger(0)
val props = Props(classOf[LifeCycleTestActor], testActor, id, gen) val props = Props(classOf[LifeCycleTestActor], testActor, id, gen)

View file

@ -249,14 +249,14 @@ class ActorLookupSpec extends AkkaSpec with DefaultTimeout {
val lookname = looker.path.elements.mkString("", "/", "/") val lookname = looker.path.elements.mkString("", "/", "/")
for ( for (
(l, r) Seq( (l, r) Seq(
LookupString("a/b/c") -> empty(lookname + "a/b/c"), LookupString("a/b/c") empty(lookname + "a/b/c"),
LookupString("") -> system.deadLetters, LookupString("") system.deadLetters,
LookupString("akka://all-systems/Nobody") -> system.deadLetters, LookupString("akka://all-systems/Nobody") system.deadLetters,
LookupPath(system / "hallo") -> empty("user/hallo"), LookupPath(system / "hallo") empty("user/hallo"),
LookupPath(looker.path child "hallo") -> empty(lookname + "hallo"), // test Java API LookupPath(looker.path child "hallo") empty(lookname + "hallo"), // test Java API
LookupPath(looker.path descendant Seq("a", "b").asJava) -> empty(lookname + "a/b"), // test Java API LookupPath(looker.path descendant Seq("a", "b").asJava) empty(lookname + "a/b"), // test Java API
LookupElems(Seq()) -> system.deadLetters, LookupElems(Seq()) system.deadLetters,
LookupElems(Seq("a")) -> empty(lookname + "a")) LookupElems(Seq("a")) empty(lookname + "a"))
) checkOne(looker, l, r) ) checkOne(looker, l, r)
} }
for (looker all) check(looker) for (looker all) check(looker)

View file

@ -210,7 +210,8 @@ object ActorMailboxSpec {
final case class MCBoundedMailbox(val capacity: Int, val pushTimeOut: FiniteDuration) final case class MCBoundedMailbox(val capacity: Int, val pushTimeOut: FiniteDuration)
extends MailboxType with ProducesMessageQueue[MCBoundedMessageQueueSemantics] { extends MailboxType with ProducesMessageQueue[MCBoundedMessageQueueSemantics] {
def this(settings: ActorSystem.Settings, config: Config) = this(config.getInt("mailbox-capacity"), def this(settings: ActorSystem.Settings, config: Config) = this(
config.getInt("mailbox-capacity"),
config.getNanosDuration("mailbox-push-timeout-time")) config.getNanosDuration("mailbox-push-timeout-time"))
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue = final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue =
@ -241,23 +242,29 @@ class ActorMailboxSpec(conf: Config) extends AkkaSpec(conf) with DefaultTimeout
} }
"get an unbounded deque message queue when it is only configured on the props" in { "get an unbounded deque message queue when it is only configured on the props" in {
checkMailboxQueue(Props[QueueReportingActor].withMailbox("akka.actor.mailbox.unbounded-deque-based"), checkMailboxQueue(
Props[QueueReportingActor].withMailbox("akka.actor.mailbox.unbounded-deque-based"),
"default-override-from-props", UnboundedDeqMailboxTypes) "default-override-from-props", UnboundedDeqMailboxTypes)
} }
"get an bounded message queue when it's only configured with RequiresMailbox" in { "get an bounded message queue when it's only configured with RequiresMailbox" in {
checkMailboxQueue(Props[BoundedQueueReportingActor], checkMailboxQueue(
Props[BoundedQueueReportingActor],
"default-override-from-trait", BoundedMailboxTypes) "default-override-from-trait", BoundedMailboxTypes)
} }
"get an unbounded deque message queue when it's only mixed with Stash" in { "get an unbounded deque message queue when it's only mixed with Stash" in {
checkMailboxQueue(Props[StashQueueReportingActor], checkMailboxQueue(
Props[StashQueueReportingActor],
"default-override-from-stash", UnboundedDeqMailboxTypes) "default-override-from-stash", UnboundedDeqMailboxTypes)
checkMailboxQueue(Props(new StashQueueReportingActor), checkMailboxQueue(
Props(new StashQueueReportingActor),
"default-override-from-stash2", UnboundedDeqMailboxTypes) "default-override-from-stash2", UnboundedDeqMailboxTypes)
checkMailboxQueue(Props(classOf[StashQueueReportingActorWithParams], 17, "hello"), checkMailboxQueue(
Props(classOf[StashQueueReportingActorWithParams], 17, "hello"),
"default-override-from-stash3", UnboundedDeqMailboxTypes) "default-override-from-stash3", UnboundedDeqMailboxTypes)
checkMailboxQueue(Props(new StashQueueReportingActorWithParams(17, "hello")), checkMailboxQueue(
Props(new StashQueueReportingActorWithParams(17, "hello")),
"default-override-from-stash4", UnboundedDeqMailboxTypes) "default-override-from-stash4", UnboundedDeqMailboxTypes)
} }
@ -278,12 +285,14 @@ class ActorMailboxSpec(conf: Config) extends AkkaSpec(conf) with DefaultTimeout
} }
"get an bounded control aware message queue when it's only configured with RequiresMailbox" in { "get an bounded control aware message queue when it's only configured with RequiresMailbox" in {
checkMailboxQueue(Props[BoundedControlAwareQueueReportingActor], checkMailboxQueue(
Props[BoundedControlAwareQueueReportingActor],
"default-override-from-trait-bounded-control-aware", BoundedControlAwareMailboxTypes) "default-override-from-trait-bounded-control-aware", BoundedControlAwareMailboxTypes)
} }
"get an unbounded control aware message queue when it's only configured with RequiresMailbox" in { "get an unbounded control aware message queue when it's only configured with RequiresMailbox" in {
checkMailboxQueue(Props[UnboundedControlAwareQueueReportingActor], checkMailboxQueue(
Props[UnboundedControlAwareQueueReportingActor],
"default-override-from-trait-unbounded-control-aware", UnboundedControlAwareMailboxTypes) "default-override-from-trait-unbounded-control-aware", UnboundedControlAwareMailboxTypes)
} }
@ -317,7 +326,8 @@ class ActorMailboxSpec(conf: Config) extends AkkaSpec(conf) with DefaultTimeout
} }
"get an unbounded message queue overriding configuration on the props" in { "get an unbounded message queue overriding configuration on the props" in {
checkMailboxQueue(Props[QueueReportingActor].withMailbox("akka.actor.mailbox.unbounded-deque-based"), checkMailboxQueue(
Props[QueueReportingActor].withMailbox("akka.actor.mailbox.unbounded-deque-based"),
"bounded-unbounded-override-props", UnboundedMailboxTypes) "bounded-unbounded-override-props", UnboundedMailboxTypes)
} }
@ -401,17 +411,20 @@ class ActorMailboxSpec(conf: Config) extends AkkaSpec(conf) with DefaultTimeout
} }
"get an unbounded message queue with a balancing dispatcher" in { "get an unbounded message queue with a balancing dispatcher" in {
checkMailboxQueue(Props[QueueReportingActor].withDispatcher("balancing-dispatcher"), checkMailboxQueue(
Props[QueueReportingActor].withDispatcher("balancing-dispatcher"),
"unbounded-balancing", UnboundedMailboxTypes) "unbounded-balancing", UnboundedMailboxTypes)
} }
"get a bounded message queue with a balancing bounded dispatcher" in { "get a bounded message queue with a balancing bounded dispatcher" in {
checkMailboxQueue(Props[QueueReportingActor].withDispatcher("balancing-bounded-dispatcher"), checkMailboxQueue(
Props[QueueReportingActor].withDispatcher("balancing-bounded-dispatcher"),
"bounded-balancing", BoundedMailboxTypes) "bounded-balancing", BoundedMailboxTypes)
} }
"get a bounded message queue with a requiring balancing bounded dispatcher" in { "get a bounded message queue with a requiring balancing bounded dispatcher" in {
checkMailboxQueue(Props[QueueReportingActor].withDispatcher("requiring-balancing-bounded-dispatcher"), checkMailboxQueue(
Props[QueueReportingActor].withDispatcher("requiring-balancing-bounded-dispatcher"),
"requiring-bounded-balancing", BoundedMailboxTypes) "requiring-bounded-balancing", BoundedMailboxTypes)
} }
} }

View file

@ -65,7 +65,8 @@ class ActorSelectionSpec extends AkkaSpec("akka.loglevel=DEBUG") with DefaultTim
asked.correlationId should ===(selection) asked.correlationId should ===(selection)
implicit val ec = system.dispatcher implicit val ec = system.dispatcher
val resolved = Await.result(selection.resolveOne(timeout.duration).mapTo[ActorRef] recover { case _ null }, val resolved = Await.result(
selection.resolveOne(timeout.duration).mapTo[ActorRef] recover { case _ null },
timeout.duration) timeout.duration)
Option(resolved) should ===(result) Option(resolved) should ===(result)
@ -248,11 +249,11 @@ class ActorSelectionSpec extends AkkaSpec("akka.loglevel=DEBUG") with DefaultTim
val lookname = looker.path.elements.mkString("", "/", "/") val lookname = looker.path.elements.mkString("", "/", "/")
for ( for (
(l, r) Seq( (l, r) Seq(
SelectString("a/b/c") -> None, SelectString("a/b/c") None,
SelectString("akka://all-systems/Nobody") -> None, SelectString("akka://all-systems/Nobody") None,
SelectPath(system / "hallo") -> None, SelectPath(system / "hallo") None,
SelectPath(looker.path child "hallo") -> None, // test Java API SelectPath(looker.path child "hallo") None, // test Java API
SelectPath(looker.path descendant Seq("a", "b").asJava) -> None) // test Java API SelectPath(looker.path descendant Seq("a", "b").asJava) None) // test Java API
) checkOne(looker, l, r) ) checkOne(looker, l, r)
} }
for (looker all) check(looker) for (looker all) check(looker)

View file

@ -273,7 +273,8 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend
} }
"allow configuration of guardian supervisor strategy" in { "allow configuration of guardian supervisor strategy" in {
implicit val system = ActorSystem("Stop", implicit val system = ActorSystem(
"Stop",
ConfigFactory.parseString("akka.actor.guardian-supervisor-strategy=akka.actor.StoppingSupervisorStrategy") ConfigFactory.parseString("akka.actor.guardian-supervisor-strategy=akka.actor.StoppingSupervisorStrategy")
.withFallback(AkkaSpec.testConf)) .withFallback(AkkaSpec.testConf))
val a = system.actorOf(Props(new Actor { val a = system.actorOf(Props(new Actor {
@ -293,7 +294,8 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend
} }
"shut down when /user escalates" in { "shut down when /user escalates" in {
implicit val system = ActorSystem("Stop", implicit val system = ActorSystem(
"Stop",
ConfigFactory.parseString("akka.actor.guardian-supervisor-strategy=\"akka.actor.ActorSystemSpec$Strategy\"") ConfigFactory.parseString("akka.actor.guardian-supervisor-strategy=\"akka.actor.ActorSystemSpec$Strategy\"")
.withFallback(AkkaSpec.testConf)) .withFallback(AkkaSpec.testConf))
val a = system.actorOf(Props(new Actor { val a = system.actorOf(Props(new Actor {

View file

@ -13,7 +13,6 @@ import org.scalatest.junit.JUnitSuiteLike
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
class JavaExtensionSpec extends JavaExtension with JUnitSuiteLike class JavaExtensionSpec extends JavaExtension with JUnitSuiteLike
object TestExtension extends ExtensionId[TestExtension] with ExtensionIdProvider { object TestExtension extends ExtensionId[TestExtension] with ExtensionIdProvider {
@ -52,7 +51,6 @@ class FailingTestExtension(val system: ExtendedActorSystem) extends Extension {
throw new FailingTestExtension.TestException throw new FailingTestExtension.TestException
} }
class ExtensionSpec extends WordSpec with Matchers { class ExtensionSpec extends WordSpec with Matchers {
"The ActorSystem extensions support" should { "The ActorSystem extensions support" should {
@ -83,7 +81,6 @@ class ExtensionSpec extends WordSpec with Matchers {
shutdownActorSystem(system) shutdownActorSystem(system)
} }
"fail the actor system if an extension listed in akka.extensions fails to start" in { "fail the actor system if an extension listed in akka.extensions fails to start" in {
intercept[RuntimeException] { intercept[RuntimeException] {
val system = ActorSystem("failing", ConfigFactory.parseString( val system = ActorSystem("failing", ConfigFactory.parseString(
@ -134,7 +131,6 @@ class ExtensionSpec extends WordSpec with Matchers {
} }
} }
} }
} }

View file

@ -34,6 +34,7 @@ object FSMActorSpec {
class Lock(code: String, timeout: FiniteDuration, latches: Latches) extends Actor with FSM[LockState, CodeState] { class Lock(code: String, timeout: FiniteDuration, latches: Latches) extends Actor with FSM[LockState, CodeState] {
import latches._ import latches._
import FSM.`→`
startWith(Locked, CodeState("", code)) startWith(Locked, CodeState("", code))
@ -71,7 +72,7 @@ object FSMActorSpec {
} }
onTransition { onTransition {
case Locked -> Open transitionLatch.open case Locked Open transitionLatch.open
} }
// verify that old-style does still compile // verify that old-style does still compile
@ -98,8 +99,9 @@ object FSMActorSpec {
final case class CodeState(soFar: String, code: String) final case class CodeState(soFar: String, code: String)
} }
class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with ImplicitSender { class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" true)) with ImplicitSender {
import FSMActorSpec._ import FSMActorSpec._
import FSM.`→`
val timeout = Timeout(2 seconds) val timeout = Timeout(2 seconds)
@ -222,7 +224,7 @@ class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with Im
case Event("stop", _) stop() case Event("stop", _) stop()
} }
onTransition { onTransition {
case "not-started" -> "started" case "not-started" "started"
for (timerName timerNames) setTimer(timerName, (), 10 seconds, false) for (timerName timerNames) setTimer(timerName, (), 10 seconds, false)
} }
onTermination { onTermination {
@ -250,8 +252,8 @@ class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with Im
"log events and transitions if asked to do so" in { "log events and transitions if asked to do so" in {
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
val config = ConfigFactory.parseMap(Map("akka.loglevel" -> "DEBUG", "akka.actor.serialize-messages" -> "off", val config = ConfigFactory.parseMap(Map("akka.loglevel" "DEBUG", "akka.actor.serialize-messages" "off",
"akka.actor.debug.fsm" -> true).asJava).withFallback(system.settings.config) "akka.actor.debug.fsm" true).asJava).withFallback(system.settings.config)
val fsmEventSystem = ActorSystem("fsmEvent", config) val fsmEventSystem = ActorSystem("fsmEvent", config)
try { try {
new TestKit(fsmEventSystem) { new TestKit(fsmEventSystem) {

View file

@ -129,7 +129,8 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender {
} }
"notify unhandled messages" taggedAs TimingTest in { "notify unhandled messages" taggedAs TimingTest in {
filterEvents(EventFilter.warning("unhandled event Tick in state TestUnhandled", source = fsm.path.toString, occurrences = 1), filterEvents(
EventFilter.warning("unhandled event Tick in state TestUnhandled", source = fsm.path.toString, occurrences = 1),
EventFilter.warning("unhandled event Unhandled(test) in state TestUnhandled", source = fsm.path.toString, occurrences = 1)) { EventFilter.warning("unhandled event Unhandled(test) in state TestUnhandled", source = fsm.path.toString, occurrences = 1)) {
fsm ! TestUnhandled fsm ! TestUnhandled
within(3 second) { within(3 second) {
@ -208,7 +209,7 @@ object FSMTimingSpec {
goto(Initial) goto(Initial)
} }
onTransition { onTransition {
case Initial -> TestSingleTimerResubmit setTimer("blah", Tick, 500.millis.dilated) case Initial TestSingleTimerResubmit setTimer("blah", Tick, 500.millis.dilated)
} }
when(TestSingleTimerResubmit) { when(TestSingleTimerResubmit) {
case Event(Tick, _) case Event(Tick, _)

View file

@ -9,6 +9,7 @@ import scala.concurrent.duration._
import scala.language.postfixOps import scala.language.postfixOps
object FSMTransitionSpec { object FSMTransitionSpec {
import FSM.`→`
class Supervisor extends Actor { class Supervisor extends Actor {
def receive = { case _ } def receive = { case _ }
@ -20,7 +21,7 @@ object FSMTransitionSpec {
case Event("stay", _) stay() case Event("stay", _) stay()
case Event(_, _) goto(0) case Event(_, _) goto(0)
} }
onTransition { case from -> to target ! (from -> to) } onTransition { case from to target ! (from to) }
initialize() initialize()
} }
@ -50,8 +51,8 @@ object FSMTransitionSpec {
case _ goto(1) case _ goto(1)
} }
onTransition { onTransition {
case 0 -> 1 target ! ((stateData, nextStateData)) case 0 1 target ! ((stateData, nextStateData))
case 1 -> 1 target ! ((stateData, nextStateData)) case 1 1 target ! ((stateData, nextStateData))
} }
} }
@ -64,16 +65,17 @@ object FSMTransitionSpec {
class FSMTransitionSpec extends AkkaSpec with ImplicitSender { class FSMTransitionSpec extends AkkaSpec with ImplicitSender {
import FSMTransitionSpec._ import FSMTransitionSpec._
import FSM.`→`
"A FSM transition notifier" must { "A FSM transition notifier" must {
"not trigger onTransition for stay" in { "not trigger onTransition for stay" in {
val fsm = system.actorOf(Props(new SendAnyTransitionFSM(testActor))) val fsm = system.actorOf(Props(new SendAnyTransitionFSM(testActor)))
expectMsg(0 -> 0) // caused by initialize(), OK. expectMsg(0 0) // caused by initialize(), OK.
fsm ! "stay" // no transition event fsm ! "stay" // no transition event
expectNoMsg(500.millis) expectNoMsg(500.millis)
fsm ! "goto" // goto(current state) fsm ! "goto" // goto(current state)
expectMsg(0 -> 0) expectMsg(0 0)
} }
"notify listeners" in { "notify listeners" in {
@ -150,7 +152,7 @@ class FSMTransitionSpec extends AkkaSpec with ImplicitSender {
case Event("switch", _) goto(1) using sender() case Event("switch", _) goto(1) using sender()
} }
onTransition { onTransition {
case x -> y nextStateData ! (x -> y) case x y nextStateData ! (x y)
} }
when(1) { when(1) {
case Event("test", _) case Event("test", _)

View file

@ -26,6 +26,8 @@ import java.lang.System.identityHashCode
import akka.util.Helpers.ConfigOps import akka.util.Helpers.ConfigOps
object SupervisorHierarchySpec { object SupervisorHierarchySpec {
import FSM.`→`
class FireWorkerException(msg: String) extends Exception(msg) class FireWorkerException(msg: String) extends Exception(msg)
/** /**
@ -79,7 +81,8 @@ object SupervisorHierarchySpec {
extends DispatcherConfigurator(config, prerequisites) { extends DispatcherConfigurator(config, prerequisites) {
private val instance: MessageDispatcher = private val instance: MessageDispatcher =
new Dispatcher(this, new Dispatcher(
this,
config.getString("id"), config.getString("id"),
config.getInt("throughput"), config.getInt("throughput"),
config.getNanosDuration("throughput-deadline-time"), config.getNanosDuration("throughput-deadline-time"),
@ -467,7 +470,7 @@ object SupervisorHierarchySpec {
} }
onTransition { onTransition {
case Init -> Stress case Init Stress
self ! Work self ! Work
idleChildren = children idleChildren = children
activeChildren = children activeChildren = children
@ -532,7 +535,7 @@ object SupervisorHierarchySpec {
} }
onTransition { onTransition {
case Stress -> Finishing ignoreFailConstr = true case Stress Finishing ignoreFailConstr = true
} }
when(Finishing) { when(Finishing) {
@ -546,7 +549,7 @@ object SupervisorHierarchySpec {
} }
onTransition { onTransition {
case _ -> LastPing case _ LastPing
idleChildren foreach (_ ! "ping") idleChildren foreach (_ ! "ping")
pingChildren ++= idleChildren pingChildren ++= idleChildren
idleChildren = Vector.empty idleChildren = Vector.empty
@ -563,7 +566,7 @@ object SupervisorHierarchySpec {
} }
onTransition { onTransition {
case _ -> Stopping case _ Stopping
ignoreNotResumedLogs = false ignoreNotResumedLogs = false
hierarchy ! PingOfDeath hierarchy ! PingOfDeath
} }
@ -596,7 +599,7 @@ object SupervisorHierarchySpec {
stop stop
} }
case Event(StateTimeout, _) case Event(StateTimeout, _)
errors :+= self -> ErrorLog("timeout while Stopping", Vector.empty) errors :+= self ErrorLog("timeout while Stopping", Vector.empty)
println(system.asInstanceOf[ActorSystemImpl].printTree) println(system.asInstanceOf[ActorSystemImpl].printTree)
getErrors(hierarchy, 10) getErrors(hierarchy, 10)
printErrors() printErrors()
@ -604,7 +607,7 @@ object SupervisorHierarchySpec {
testActor ! "timeout in Stopping" testActor ! "timeout in Stopping"
stop stop
case Event(e: ErrorLog, _) case Event(e: ErrorLog, _)
errors :+= sender() -> e errors :+= sender() e
goto(Failed) goto(Failed)
} }
@ -630,7 +633,7 @@ object SupervisorHierarchySpec {
when(Failed, stateTimeout = 5.seconds.dilated) { when(Failed, stateTimeout = 5.seconds.dilated) {
case Event(e: ErrorLog, _) case Event(e: ErrorLog, _)
if (!e.msg.startsWith("not resumed") || !ignoreNotResumedLogs) if (!e.msg.startsWith("not resumed") || !ignoreNotResumedLogs)
errors :+= sender() -> e errors :+= sender() e
stay stay
case Event(Terminated(r), _) if r == hierarchy case Event(Terminated(r), _) if r == hierarchy
printErrors() printErrors()
@ -650,8 +653,8 @@ object SupervisorHierarchySpec {
target match { target match {
case l: LocalActorRef case l: LocalActorRef
l.underlying.actor match { l.underlying.actor match {
case h: Hierarchy errors :+= target -> ErrorLog("forced", h.log) case h: Hierarchy errors :+= target ErrorLog("forced", h.log)
case _ errors :+= target -> ErrorLog("fetched", stateCache.get(target.path).log) case _ errors :+= target ErrorLog("fetched", stateCache.get(target.path).log)
} }
if (depth > 0) { if (depth > 0) {
l.underlying.children foreach (getErrors(_, depth - 1)) l.underlying.children foreach (getErrors(_, depth - 1))
@ -663,8 +666,8 @@ object SupervisorHierarchySpec {
target match { target match {
case l: LocalActorRef case l: LocalActorRef
l.underlying.actor match { l.underlying.actor match {
case h: Hierarchy errors :+= target -> ErrorLog("forced", h.log) case h: Hierarchy errors :+= target ErrorLog("forced", h.log)
case _ errors :+= target -> ErrorLog("fetched", stateCache.get(target.path).log) case _ errors :+= target ErrorLog("fetched", stateCache.get(target.path).log)
} }
if (target != hierarchy) getErrorsUp(l.getParent) if (target != hierarchy) getErrorsUp(l.getParent)
} }
@ -693,7 +696,7 @@ object SupervisorHierarchySpec {
case Event(e: ErrorLog, _) case Event(e: ErrorLog, _)
if (e.msg.startsWith("not resumed")) stay if (e.msg.startsWith("not resumed")) stay
else { else {
errors :+= sender() -> e errors :+= sender() e
// dont stop the hierarchy, that is going to happen all by itself and in the right order // dont stop the hierarchy, that is going to happen all by itself and in the right order
goto(Failed) goto(Failed)
} }

View file

@ -58,7 +58,7 @@ class SupervisorMiscSpec extends AkkaSpec(SupervisorMiscSpec.config) with Defaul
countDownLatch.await(10, TimeUnit.SECONDS) countDownLatch.await(10, TimeUnit.SECONDS)
Seq("actor1" -> actor1, "actor2" -> actor2, "actor3" -> actor3, "actor4" -> actor4) map { Seq("actor1" actor1, "actor2" actor2, "actor3" actor3, "actor4" actor4) map {
case (id, ref) (id, ref ? "status") case (id, ref) (id, ref ? "status")
} foreach { } foreach {
case (id, f) (id, Await.result(f, timeout.duration)) should ===((id, "OK")) case (id, f) (id, Await.result(f, timeout.duration)) should ===((id, "OK"))

View file

@ -16,7 +16,8 @@ object UidClashTest {
@volatile var oldActor: ActorRef = _ @volatile var oldActor: ActorRef = _
private[akka] class EvilCollidingActorRef(override val provider: ActorRefProvider, private[akka] class EvilCollidingActorRef(
override val provider: ActorRefProvider,
override val path: ActorPath, override val path: ActorPath,
val eventStream: EventStream) extends MinimalActorRef { val eventStream: EventStream) extends MinimalActorRef {

View file

@ -218,7 +218,8 @@ object ActorModelSpec {
await(deadline)(stats.restarts.get() == restarts) await(deadline)(stats.restarts.get() == restarts)
} catch { } catch {
case e: Throwable case e: Throwable
system.eventStream.publish(Error(e, system.eventStream.publish(Error(
e,
Option(dispatcher).toString, Option(dispatcher).toString,
(Option(dispatcher) getOrElse this).getClass, (Option(dispatcher) getOrElse this).getClass,
"actual: " + stats + ", required: InterceptorStats(susp=" + suspensions + "actual: " + stats + ", required: InterceptorStats(susp=" + suspensions +
@ -529,7 +530,8 @@ object DispatcherModelSpec {
import akka.util.Helpers.ConfigOps import akka.util.Helpers.ConfigOps
private val instance: MessageDispatcher = private val instance: MessageDispatcher =
new Dispatcher(this, new Dispatcher(
this,
config.getString("id"), config.getString("id"),
config.getInt("throughput"), config.getInt("throughput"),
config.getNanosDuration("throughput-deadline-time"), config.getNanosDuration("throughput-deadline-time"),
@ -602,7 +604,8 @@ object BalancingDispatcherModelSpec {
import akka.util.Helpers.ConfigOps import akka.util.Helpers.ConfigOps
override protected def create(mailboxType: MailboxType): BalancingDispatcher = override protected def create(mailboxType: MailboxType): BalancingDispatcher =
new BalancingDispatcher(this, new BalancingDispatcher(
this,
config.getString("id"), config.getString("id"),
config.getInt("throughput"), config.getInt("throughput"),
config.getNanosDuration("throughput-deadline-time"), config.getNanosDuration("throughput-deadline-time"),

View file

@ -104,15 +104,15 @@ class DispatchersSpec extends AkkaSpec(DispatchersSpec.config) with ImplicitSend
def ofType[T <: MessageDispatcher: ClassTag]: (MessageDispatcher) Boolean = _.getClass == implicitly[ClassTag[T]].runtimeClass def ofType[T <: MessageDispatcher: ClassTag]: (MessageDispatcher) Boolean = _.getClass == implicitly[ClassTag[T]].runtimeClass
def typesAndValidators: Map[String, (MessageDispatcher) Boolean] = Map( def typesAndValidators: Map[String, (MessageDispatcher) Boolean] = Map(
"PinnedDispatcher" -> ofType[PinnedDispatcher], "PinnedDispatcher" ofType[PinnedDispatcher],
"Dispatcher" -> ofType[Dispatcher]) "Dispatcher" ofType[Dispatcher])
def validTypes = typesAndValidators.keys.toList def validTypes = typesAndValidators.keys.toList
val defaultDispatcherConfig = settings.config.getConfig("akka.actor.default-dispatcher") val defaultDispatcherConfig = settings.config.getConfig("akka.actor.default-dispatcher")
lazy val allDispatchers: Map[String, MessageDispatcher] = { lazy val allDispatchers: Map[String, MessageDispatcher] = {
validTypes.map(t (t, from(ConfigFactory.parseMap(Map(tipe -> t, id -> t).asJava). validTypes.map(t (t, from(ConfigFactory.parseMap(Map(tipe t, id t).asJava).
withFallback(defaultDispatcherConfig)))).toMap withFallback(defaultDispatcherConfig)))).toMap
} }
@ -150,7 +150,7 @@ class DispatchersSpec extends AkkaSpec(DispatchersSpec.config) with ImplicitSend
"throw ConfigurationException if type does not exist" in { "throw ConfigurationException if type does not exist" in {
intercept[ConfigurationException] { intercept[ConfigurationException] {
from(ConfigFactory.parseMap(Map(tipe -> "typedoesntexist", id -> "invalid-dispatcher").asJava). from(ConfigFactory.parseMap(Map(tipe "typedoesntexist", id "invalid-dispatcher").asJava).
withFallback(defaultDispatcherConfig)) withFallback(defaultDispatcherConfig))
} }
} }

View file

@ -125,14 +125,16 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn
q.hasMessages should ===(false) q.hasMessages should ===(false)
} }
def testEnqueueDequeue(config: MailboxType, def testEnqueueDequeue(
config: MailboxType,
enqueueN: Int = 10000, enqueueN: Int = 10000,
dequeueN: Int = 10000, dequeueN: Int = 10000,
parallel: Boolean = true): Unit = within(10 seconds) { parallel: Boolean = true): Unit = within(10 seconds) {
val q = factory(config) val q = factory(config)
ensureInitialMailboxState(config, q) ensureInitialMailboxState(config, q)
EventFilter.warning(pattern = ".*received dead letter from Actor.*MailboxSpec/deadLetters.*", EventFilter.warning(
pattern = ".*received dead letter from Actor.*MailboxSpec/deadLetters.*",
occurrences = (enqueueN - dequeueN)) intercept { occurrences = (enqueueN - dequeueN)) intercept {
def createProducer(fromNum: Int, toNum: Int): Future[Vector[Envelope]] = spawn { def createProducer(fromNum: Int, toNum: Int): Future[Vector[Envelope]] = spawn {

View file

@ -117,10 +117,10 @@ object LoggerSpec {
override def mdc(currentMessage: Any): MDC = { override def mdc(currentMessage: Any): MDC = {
reqId += 1 reqId += 1
val always = Map("requestId" -> reqId) val always = Map("requestId" reqId)
val cmim = "Current Message in MDC" val cmim = "Current Message in MDC"
val perMessage = currentMessage match { val perMessage = currentMessage match {
case `cmim` Map[String, Any]("currentMsg" -> cmim, "currentMsgLength" -> cmim.length) case `cmim` Map[String, Any]("currentMsg" cmim, "currentMsgLength" cmim.length)
case _ Map() case _ Map()
} }
always ++ perMessage always ++ perMessage

View file

@ -28,9 +28,9 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterAll {
akka.loglevel=DEBUG akka.loglevel=DEBUG
akka.actor.serialize-messages = off # debug noise from serialization akka.actor.serialize-messages = off # debug noise from serialization
""").withFallback(AkkaSpec.testConf) """).withFallback(AkkaSpec.testConf)
val appLogging = ActorSystem("logging", ConfigFactory.parseMap(Map("akka.actor.debug.receive" -> true).asJava).withFallback(config)) val appLogging = ActorSystem("logging", ConfigFactory.parseMap(Map("akka.actor.debug.receive" true).asJava).withFallback(config))
val appAuto = ActorSystem("autoreceive", ConfigFactory.parseMap(Map("akka.actor.debug.autoreceive" -> true).asJava).withFallback(config)) val appAuto = ActorSystem("autoreceive", ConfigFactory.parseMap(Map("akka.actor.debug.autoreceive" true).asJava).withFallback(config))
val appLifecycle = ActorSystem("lifecycle", ConfigFactory.parseMap(Map("akka.actor.debug.lifecycle" -> true).asJava).withFallback(config)) val appLifecycle = ActorSystem("lifecycle", ConfigFactory.parseMap(Map("akka.actor.debug.lifecycle" true).asJava).withFallback(config))
val filter = TestEvent.Mute(EventFilter.custom { val filter = TestEvent.Mute(EventFilter.custom {
case _: Logging.Debug true case _: Logging.Debug true

View file

@ -360,11 +360,10 @@ class TcpConnectionSpec extends AkkaSpec("""
"respect pull mode" in new EstablishedConnectionTest(pullMode = true) { "respect pull mode" in new EstablishedConnectionTest(pullMode = true) {
// override config to decrease default buffer size // override config to decrease default buffer size
val config = def config =
ConfigFactory.load(
ConfigFactory.parseString("akka.io.tcp.direct-buffer-size = 1k") ConfigFactory.parseString("akka.io.tcp.direct-buffer-size = 1k")
.withFallback(AkkaSpec.testConf)) .withFallback(AkkaSpec.testConf)
override implicit def system: ActorSystem = ActorSystem("respectPullModeTest", config) override implicit lazy val system: ActorSystem = ActorSystem("respectPullModeTest", config)
try run { try run {
val maxBufferSize = 1 * 1024 val maxBufferSize = 1 * 1024
@ -402,7 +401,7 @@ class TcpConnectionSpec extends AkkaSpec("""
connectionHandler.expectMsgType[Received].data.decodeString("ASCII") should ===(vs) connectionHandler.expectMsgType[Received].data.decodeString("ASCII") should ===(vs)
} }
finally system.terminate() finally shutdown(system)
} }
"close the connection and reply with `Closed` upon reception of a `Close` command" in "close the connection and reply with `Closed` upon reception of a `Close` command" in
@ -887,7 +886,8 @@ class TcpConnectionSpec extends AkkaSpec("""
def setServerSocketOptions() = () def setServerSocketOptions() = ()
def createConnectionActor(serverAddress: InetSocketAddress = serverAddress, def createConnectionActor(
serverAddress: InetSocketAddress = serverAddress,
options: immutable.Seq[SocketOption] = Nil, options: immutable.Seq[SocketOption] = Nil,
timeout: Option[FiniteDuration] = None, timeout: Option[FiniteDuration] = None,
pullMode: Boolean = false): TestActorRef[TcpOutgoingConnection] = { pullMode: Boolean = false): TestActorRef[TcpOutgoingConnection] = {
@ -902,7 +902,8 @@ class TcpConnectionSpec extends AkkaSpec("""
def disableInterest(op: Int): Unit = interestCallReceiver.ref ! -op def disableInterest(op: Int): Unit = interestCallReceiver.ref ! -op
} }
def createConnectionActorWithoutRegistration(serverAddress: InetSocketAddress = serverAddress, def createConnectionActorWithoutRegistration(
serverAddress: InetSocketAddress = serverAddress,
options: immutable.Seq[SocketOption] = Nil, options: immutable.Seq[SocketOption] = Nil,
timeout: Option[FiniteDuration] = None, timeout: Option[FiniteDuration] = None,
pullMode: Boolean = false): TestActorRef[TcpOutgoingConnection] = pullMode: Boolean = false): TestActorRef[TcpOutgoingConnection] =
@ -1075,7 +1076,7 @@ class TcpConnectionSpec extends AkkaSpec("""
} }
val interestsNames = val interestsNames =
Seq(OP_ACCEPT -> "accepting", OP_CONNECT -> "connecting", OP_READ -> "reading", OP_WRITE -> "writing") Seq(OP_ACCEPT "accepting", OP_CONNECT "connecting", OP_READ "reading", OP_WRITE "writing")
def interestsDesc(interests: Int): String = def interestsDesc(interests: Int): String =
interestsNames.filter(i (i._1 & interests) != 0).map(_._2).mkString(", ") interestsNames.filter(i (i._1 & interests) != 0).map(_._2).mkString(", ")

View file

@ -213,7 +213,7 @@ class AskSpec extends AkkaSpec {
val act = system.actorOf(Props(new Actor { val act = system.actorOf(Props(new Actor {
def receive = { def receive = {
case msg p.ref ! sender() -> msg case msg p.ref ! sender() msg
} }
})) }))

View file

@ -44,7 +44,8 @@ object MetricsBasedResizerSpec {
var msgs: Set[TestLatch] = Set() var msgs: Set[TestLatch] = Set()
def mockSend(await: Boolean, def mockSend(
await: Boolean,
l: TestLatch = TestLatch(), l: TestLatch = TestLatch(),
routeeIdx: Int = Random.nextInt(routees.length)): Latches = { routeeIdx: Int = Random.nextInt(routees.length)): Latches = {
val target = routees(routeeIdx) val target = routees(routeeIdx)

View file

@ -50,7 +50,7 @@ class RandomSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
val counter = new AtomicInteger val counter = new AtomicInteger
var replies = Map.empty[Int, Int] var replies = Map.empty[Int, Int]
for (i 0 until connectionCount) { for (i 0 until connectionCount) {
replies = replies + (i -> 0) replies = replies + (i 0)
} }
val actor = system.actorOf(RandomPool(connectionCount).props(routeeProps = val actor = system.actorOf(RandomPool(connectionCount).props(routeeProps =
@ -65,7 +65,7 @@ class RandomSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
for (i 0 until iterationCount) { for (i 0 until iterationCount) {
for (k 0 until connectionCount) { for (k 0 until connectionCount) {
val id = Await.result((actor ? "hit").mapTo[Int], timeout.duration) val id = Await.result((actor ? "hit").mapTo[Int], timeout.duration)
replies = replies + (id -> (replies(id) + 1)) replies = replies + (id (replies(id) + 1))
} }
} }

View file

@ -64,7 +64,7 @@ class RoundRobinSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
for (_ 1 to iterationCount; _ 1 to connectionCount) { for (_ 1 to iterationCount; _ 1 to connectionCount) {
val id = Await.result((actor ? "hit").mapTo[Int], timeout.duration) val id = Await.result((actor ? "hit").mapTo[Int], timeout.duration)
replies = replies + (id -> (replies(id) + 1)) replies = replies + (id (replies(id) + 1))
} }
counter.get should ===(connectionCount) counter.get should ===(connectionCount)
@ -138,7 +138,7 @@ class RoundRobinSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
for (_ 1 to iterationCount; _ 1 to connectionCount) { for (_ 1 to iterationCount; _ 1 to connectionCount) {
val id = Await.result((actor ? "hit").mapTo[String], timeout.duration) val id = Await.result((actor ? "hit").mapTo[String], timeout.duration)
replies = replies + (id -> (replies(id) + 1)) replies = replies + (id (replies(id) + 1))
} }
actor ! akka.routing.Broadcast("end") actor ! akka.routing.Broadcast("end")
@ -184,7 +184,7 @@ class RoundRobinSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
for (_ 1 to iterationCount; _ 1 to connectionCount) { for (_ 1 to iterationCount; _ 1 to connectionCount) {
val id = Await.result((actor ? "hit").mapTo[String], timeout.duration) val id = Await.result((actor ? "hit").mapTo[String], timeout.duration)
replies = replies + (id -> (replies(id) + 1)) replies = replies + (id (replies(id) + 1))
} }
watch(actor) watch(actor)

View file

@ -342,7 +342,8 @@ class SerializationCompatibilitySpec extends AkkaSpec(SerializationTests.mostlyR
"be preserved for the Create SystemMessage" in { "be preserved for the Create SystemMessage" in {
// Using null as the cause to avoid a large serialized message and JDK differences // Using null as the cause to avoid a large serialized message and JDK differences
verify(Create(Some(null)), verify(
Create(Some(null)),
if (scala.util.Properties.versionNumberString.startsWith("2.10.")) { if (scala.util.Properties.versionNumberString.startsWith("2.10.")) {
"aced00057372001b616b6b612e64697370617463682e7379736d73672e4372656174650000000000" + "aced00057372001b616b6b612e64697370617463682e7379736d73672e4372656174650000000000" +
"0000010200014c00076661696c75726574000e4c7363616c612f4f7074696f6e3b78707372000a73" + "0000010200014c00076661696c75726574000e4c7363616c612f4f7074696f6e3b78707372000a73" +
@ -356,53 +357,62 @@ class SerializationCompatibilitySpec extends AkkaSpec(SerializationTests.mostlyR
}) })
} }
"be preserved for the Recreate SystemMessage" in { "be preserved for the Recreate SystemMessage" in {
verify(Recreate(null), verify(
Recreate(null),
"aced00057372001d616b6b612e64697370617463682e7379736d73672e5265637265617465000000" + "aced00057372001d616b6b612e64697370617463682e7379736d73672e5265637265617465000000" +
"00000000010200014c000563617573657400154c6a6176612f6c616e672f5468726f7761626c653b" + "00000000010200014c000563617573657400154c6a6176612f6c616e672f5468726f7761626c653b" +
"787070") "787070")
} }
"be preserved for the Suspend SystemMessage" in { "be preserved for the Suspend SystemMessage" in {
verify(Suspend(), verify(
Suspend(),
"aced00057372001c616b6b612e64697370617463682e7379736d73672e53757370656e6400000000" + "aced00057372001c616b6b612e64697370617463682e7379736d73672e53757370656e6400000000" +
"000000010200007870") "000000010200007870")
} }
"be preserved for the Resume SystemMessage" in { "be preserved for the Resume SystemMessage" in {
verify(Resume(null), verify(
Resume(null),
"aced00057372001b616b6b612e64697370617463682e7379736d73672e526573756d650000000000" + "aced00057372001b616b6b612e64697370617463682e7379736d73672e526573756d650000000000" +
"0000010200014c000f63617573656442794661696c7572657400154c6a6176612f6c616e672f5468" + "0000010200014c000f63617573656442794661696c7572657400154c6a6176612f6c616e672f5468" +
"726f7761626c653b787070") "726f7761626c653b787070")
} }
"be preserved for the Terminate SystemMessage" in { "be preserved for the Terminate SystemMessage" in {
verify(Terminate(), verify(
Terminate(),
"aced00057372001e616b6b612e64697370617463682e7379736d73672e5465726d696e6174650000" + "aced00057372001e616b6b612e64697370617463682e7379736d73672e5465726d696e6174650000" +
"0000000000010200007870") "0000000000010200007870")
} }
"be preserved for the Supervise SystemMessage" in { "be preserved for the Supervise SystemMessage" in {
verify(Supervise(null, true), verify(
Supervise(null, true),
"aced00057372001e616b6b612e64697370617463682e7379736d73672e5375706572766973650000" + "aced00057372001e616b6b612e64697370617463682e7379736d73672e5375706572766973650000" +
"0000000000010200025a00056173796e634c00056368696c647400154c616b6b612f6163746f722f" + "0000000000010200025a00056173796e634c00056368696c647400154c616b6b612f6163746f722f" +
"4163746f725265663b78700170") "4163746f725265663b78700170")
} }
"be preserved for the Watch SystemMessage" in { "be preserved for the Watch SystemMessage" in {
verify(Watch(null, null), verify(
Watch(null, null),
"aced00057372001a616b6b612e64697370617463682e7379736d73672e5761746368000000000000" + "aced00057372001a616b6b612e64697370617463682e7379736d73672e5761746368000000000000" +
"00010200024c00077761746368656574001d4c616b6b612f6163746f722f496e7465726e616c4163" + "00010200024c00077761746368656574001d4c616b6b612f6163746f722f496e7465726e616c4163" +
"746f725265663b4c00077761746368657271007e000178707070") "746f725265663b4c00077761746368657271007e000178707070")
} }
"be preserved for the Unwatch SystemMessage" in { "be preserved for the Unwatch SystemMessage" in {
verify(Unwatch(null, null), verify(
Unwatch(null, null),
"aced00057372001c616b6b612e64697370617463682e7379736d73672e556e776174636800000000" + "aced00057372001c616b6b612e64697370617463682e7379736d73672e556e776174636800000000" +
"000000010200024c0007776174636865657400154c616b6b612f6163746f722f4163746f72526566" + "000000010200024c0007776174636865657400154c616b6b612f6163746f722f4163746f72526566" +
"3b4c00077761746368657271007e000178707070") "3b4c00077761746368657271007e000178707070")
} }
"be preserved for the NoMessage SystemMessage" in { "be preserved for the NoMessage SystemMessage" in {
verify(NoMessage, verify(
NoMessage,
"aced00057372001f616b6b612e64697370617463682e7379736d73672e4e6f4d6573736167652400" + "aced00057372001f616b6b612e64697370617463682e7379736d73672e4e6f4d6573736167652400" +
"000000000000010200007870") "000000000000010200007870")
} }
"be preserved for the Failed SystemMessage" in { "be preserved for the Failed SystemMessage" in {
// Using null as the cause to avoid a large serialized message and JDK differences // Using null as the cause to avoid a large serialized message and JDK differences
verify(Failed(null, cause = null, uid = 0), verify(
Failed(null, cause = null, uid = 0),
"aced00057372001b616b6b612e64697370617463682e7379736d73672e4661696c65640000000000" + "aced00057372001b616b6b612e64697370617463682e7379736d73672e4661696c65640000000000" +
"0000010200034900037569644c000563617573657400154c6a6176612f6c616e672f5468726f7761" + "0000010200034900037569644c000563617573657400154c6a6176612f6c616e672f5468726f7761" +
"626c653b4c00056368696c647400154c616b6b612f6163746f722f4163746f725265663b78700000" + "626c653b4c00056368696c647400154c616b6b612f6163746f722f4163746f725265663b78700000" +

View file

@ -121,7 +121,7 @@ class ByteStringSpec extends WordSpec with Matchers with Checkers {
val (bsAIt, bsBIt) = (a.iterator, b.iterator) val (bsAIt, bsBIt) = (a.iterator, b.iterator)
val (vecAIt, vecBIt) = (Vector(a: _*).iterator.buffered, Vector(b: _*).iterator.buffered) val (vecAIt, vecBIt) = (Vector(a: _*).iterator.buffered, Vector(b: _*).iterator.buffered)
(body(bsAIt, bsBIt) == body(vecAIt, vecBIt)) && (body(bsAIt, bsBIt) == body(vecAIt, vecBIt)) &&
(!strict || (bsAIt.toSeq -> bsBIt.toSeq) == (vecAIt.toSeq -> vecBIt.toSeq)) (!strict || (bsAIt.toSeq bsBIt.toSeq) == (vecAIt.toSeq vecBIt.toSeq))
} }
def likeVecBld(body: Builder[Byte, _] Unit): Boolean = { def likeVecBld(body: Builder[Byte, _] Unit): Boolean = {

View file

@ -15,16 +15,16 @@ class PrettyDurationSpec extends FlatSpec with Matchers {
import scala.concurrent.duration._ import scala.concurrent.duration._
val cases: Seq[(Duration, String)] = val cases: Seq[(Duration, String)] =
9.nanos -> "9.000 ns" :: 9.nanos "9.000 ns" ::
95.nanos -> "95.00 ns" :: 95.nanos "95.00 ns" ::
999.nanos -> "999.0 ns" :: 999.nanos "999.0 ns" ::
1000.nanos -> "1.000 μs" :: 1000.nanos "1.000 μs" ::
9500.nanos -> "9.500 μs" :: 9500.nanos "9.500 μs" ::
9500.micros -> "9.500 ms" :: 9500.micros "9.500 ms" ::
9500.millis -> "9.500 s" :: 9500.millis "9.500 s" ::
95.seconds -> "1.583 min" :: 95.seconds "1.583 min" ::
95.minutes -> "1.583 h" :: 95.minutes "1.583 h" ::
95.hours -> "3.958 d" :: 95.hours "3.958 d" ::
Nil Nil
cases foreach { cases foreach {

View file

@ -66,7 +66,8 @@ abstract class AbstractFSM[S, D] extends FSM[S, D] {
* @param stateTimeout default state timeout for this state * @param stateTimeout default state timeout for this state
* @param stateFunctionBuilder partial function builder describing response to input * @param stateFunctionBuilder partial function builder describing response to input
*/ */
final def when(stateName: S, final def when(
stateName: S,
stateTimeout: FiniteDuration, stateTimeout: FiniteDuration,
stateFunctionBuilder: FSMStateFunctionBuilder[S, D]): Unit = stateFunctionBuilder: FSMStateFunctionBuilder[S, D]): Unit =
when(stateName, stateTimeout)(stateFunctionBuilder.build()) when(stateName, stateTimeout)(stateFunctionBuilder.build())

View file

@ -87,7 +87,7 @@ private[akka] trait AbstractProps {
if (i == declaredConstructors.length) false if (i == declaredConstructors.length) false
else { else {
val c = declaredConstructors(i) val c = declaredConstructors(i)
if (c.getParameterCount >= 1 && c.getParameterTypes()(i) == enclosingClass) if (c.getParameterCount >= 1 && c.getParameterTypes()(0) == enclosingClass)
true true
else else
loop(i + 1) // recur loop(i + 1) // recur

View file

@ -189,7 +189,8 @@ object ActorInitializationException {
*/ */
@SerialVersionUID(1L) @SerialVersionUID(1L)
final case class PreRestartException private[akka] (actor: ActorRef, cause: Throwable, originalCause: Throwable, messageOption: Option[Any]) final case class PreRestartException private[akka] (actor: ActorRef, cause: Throwable, originalCause: Throwable, messageOption: Option[Any])
extends ActorInitializationException(actor, extends ActorInitializationException(
actor,
"exception in preRestart(" + "exception in preRestart(" +
(if (originalCause == null) "null" else originalCause.getClass) + ", " + (if (originalCause == null) "null" else originalCause.getClass) + ", " +
(messageOption match { case Some(m: AnyRef) m.getClass; case _ "None" }) + (messageOption match { case Some(m: AnyRef) m.getClass; case _ "None" }) +
@ -205,7 +206,8 @@ final case class PreRestartException private[akka] (actor: ActorRef, cause: Thro
*/ */
@SerialVersionUID(1L) @SerialVersionUID(1L)
final case class PostRestartException private[akka] (actor: ActorRef, cause: Throwable, originalCause: Throwable) final case class PostRestartException private[akka] (actor: ActorRef, cause: Throwable, originalCause: Throwable)
extends ActorInitializationException(actor, extends ActorInitializationException(
actor,
"exception post restart (" + (if (originalCause == null) "null" else originalCause.getClass) + ")", cause) "exception post restart (" + (if (originalCause == null) "null" else originalCause.getClass) + ")", cause)
/** /**

View file

@ -598,7 +598,8 @@ private[akka] class ActorCell(
case NonFatal(e) case NonFatal(e)
clearOutActorIfNonNull() clearOutActorIfNonNull()
e match { e match {
case i: InstantiationException throw ActorInitializationException(self, case i: InstantiationException throw ActorInitializationException(
self,
"""exception during creation, this problem is likely to occur because the class of the Actor you tried to create is either, """exception during creation, this problem is likely to occur because the class of the Actor you tried to create is either,
a non-static inner class (in which case make it a static inner class or use Props(new ...) or Props( new Creator ... ) a non-static inner class (in which case make it a static inner class or use Props(new ...) or Props( new Creator ... )
or is missing an appropriate, reachable no-args constructor. or is missing an appropriate, reachable no-args constructor.

View file

@ -254,7 +254,8 @@ sealed trait ActorPath extends Comparable[ActorPath] with Serializable {
*/ */
@SerialVersionUID(1L) @SerialVersionUID(1L)
final case class RootActorPath(address: Address, name: String = "/") extends ActorPath { final case class RootActorPath(address: Address, name: String = "/") extends ActorPath {
require(name.length == 1 || name.indexOf('/', 1) == -1, require(
name.length == 1 || name.indexOf('/', 1) == -1,
"/ may only exist at the beginning of the root actors name, " + "/ may only exist at the beginning of the root actors name, " +
"it is a path separator and is not legal in ActorPath names: [%s]" format name) "it is a path separator and is not legal in ActorPath names: [%s]" format name)
require(name.indexOf('#') == -1, "# is a fragment separator and is not legal in ActorPath names: [%s]" format name) require(name.indexOf('#') == -1, "# is a fragment separator and is not legal in ActorPath names: [%s]" format name)

View file

@ -518,7 +518,8 @@ private[akka] object DeadLetterActorRef {
* *
* INTERNAL API * INTERNAL API
*/ */
private[akka] class EmptyLocalActorRef(override val provider: ActorRefProvider, private[akka] class EmptyLocalActorRef(
override val provider: ActorRefProvider,
override val path: ActorPath, override val path: ActorPath,
val eventStream: EventStream) extends MinimalActorRef { val eventStream: EventStream) extends MinimalActorRef {
@ -570,7 +571,8 @@ private[akka] class EmptyLocalActorRef(override val provider: ActorRefProvider,
* *
* INTERNAL API * INTERNAL API
*/ */
private[akka] class DeadLetterActorRef(_provider: ActorRefProvider, private[akka] class DeadLetterActorRef(
_provider: ActorRefProvider,
_path: ActorPath, _path: ActorPath,
_eventStream: EventStream) extends EmptyLocalActorRef(_provider, _path, _eventStream) { _eventStream: EventStream) extends EmptyLocalActorRef(_provider, _path, _eventStream) {
@ -705,7 +707,8 @@ private[akka] class VirtualPathContainer(
* When using the watch() feature you must ensure that upon reception of the * When using the watch() feature you must ensure that upon reception of the
* Terminated message the watched actorRef is unwatch()ed. * Terminated message the watched actorRef is unwatch()ed.
*/ */
private[akka] final class FunctionRef(override val path: ActorPath, private[akka] final class FunctionRef(
override val path: ActorPath,
override val provider: ActorRefProvider, override val provider: ActorRefProvider,
val eventStream: EventStream, val eventStream: EventStream,
f: (ActorRef, Any) Unit) extends MinimalActorRef { f: (ActorRef, Any) Unit) extends MinimalActorRef {

View file

@ -484,11 +484,13 @@ private[akka] class LocalActorRefProvider private[akka] (
extends ActorRefProvider { extends ActorRefProvider {
// this is the constructor needed for reflectively instantiating the provider // this is the constructor needed for reflectively instantiating the provider
def this(_systemName: String, def this(
_systemName: String,
settings: ActorSystem.Settings, settings: ActorSystem.Settings,
eventStream: EventStream, eventStream: EventStream,
dynamicAccess: DynamicAccess) = dynamicAccess: DynamicAccess) =
this(_systemName, this(
_systemName,
settings, settings,
eventStream, eventStream,
dynamicAccess, dynamicAccess,
@ -776,7 +778,8 @@ private[akka] class LocalActorRefProvider private[akka] (
if (!system.dispatchers.hasDispatcher(r.routerDispatcher)) if (!system.dispatchers.hasDispatcher(r.routerDispatcher))
throw new ConfigurationException(s"Dispatcher [${p.dispatcher}] not configured for router of $path") throw new ConfigurationException(s"Dispatcher [${p.dispatcher}] not configured for router of $path")
val routerProps = Props(p.deploy.copy(dispatcher = p.routerConfig.routerDispatcher), val routerProps = Props(
p.deploy.copy(dispatcher = p.routerConfig.routerDispatcher),
classOf[RoutedActorCell.RouterActorCreator], Vector(p.routerConfig)) classOf[RoutedActorCell.RouterActorCreator], Vector(p.routerConfig))
val routeeProps = p.withRouter(NoRouter) val routeeProps = p.withRouter(NoRouter)

View file

@ -218,7 +218,8 @@ object ActorSelection {
if (matchingChildren.isEmpty && !sel.wildcardFanOut) if (matchingChildren.isEmpty && !sel.wildcardFanOut)
emptyRef.tell(sel, sender) emptyRef.tell(sel, sender)
else { else {
val m = sel.copy(elements = iter.toVector, val m = sel.copy(
elements = iter.toVector,
wildcardFanOut = sel.wildcardFanOut || matchingChildren.size > 1) wildcardFanOut = sel.wildcardFanOut || matchingChildren.size > 1)
matchingChildren.foreach(c deliverSelection(c.asInstanceOf[InternalActorRef], sender, m)) matchingChildren.foreach(c deliverSelection(c.asInstanceOf[InternalActorRef], sender, m))
} }

View file

@ -593,7 +593,7 @@ private[akka] class ActorSystemImpl(
eventStream.startStdoutLogger(settings) eventStream.startStdoutLogger(settings)
val logFilter: LoggingFilter = { val logFilter: LoggingFilter = {
val arguments = Vector(classOf[Settings] -> settings, classOf[EventStream] -> eventStream) val arguments = Vector(classOf[Settings] settings, classOf[EventStream] eventStream)
dynamicAccess.createInstanceFor[LoggingFilter](LoggingFilter, arguments).get dynamicAccess.createInstanceFor[LoggingFilter](LoggingFilter, arguments).get
} }
@ -603,10 +603,10 @@ private[akka] class ActorSystemImpl(
val provider: ActorRefProvider = try { val provider: ActorRefProvider = try {
val arguments = Vector( val arguments = Vector(
classOf[String] -> name, classOf[String] name,
classOf[Settings] -> settings, classOf[Settings] settings,
classOf[EventStream] -> eventStream, classOf[EventStream] eventStream,
classOf[DynamicAccess] -> dynamicAccess) classOf[DynamicAccess] dynamicAccess)
dynamicAccess.createInstanceFor[ActorRefProvider](ProviderClass, arguments).get dynamicAccess.createInstanceFor[ActorRefProvider](ProviderClass, arguments).get
} catch { } catch {
@ -698,9 +698,9 @@ private[akka] class ActorSystemImpl(
*/ */
protected def createScheduler(): Scheduler = protected def createScheduler(): Scheduler =
dynamicAccess.createInstanceFor[Scheduler](settings.SchedulerClass, immutable.Seq( dynamicAccess.createInstanceFor[Scheduler](settings.SchedulerClass, immutable.Seq(
classOf[Config] -> settings.config, classOf[Config] settings.config,
classOf[LoggingAdapter] -> log, classOf[LoggingAdapter] log,
classOf[ThreadFactory] -> threadFactory.withName(threadFactory.name + "-scheduler"))).get classOf[ThreadFactory] threadFactory.withName(threadFactory.name + "-scheduler"))).get
//#create-scheduler //#create-scheduler
/* /*

View file

@ -137,7 +137,7 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce
protected val default = config.getConfig("default") protected val default = config.getConfig("default")
val routerTypeMapping: Map[String, String] = val routerTypeMapping: Map[String, String] =
settings.config.getConfig("akka.actor.router.type-mapping").root.unwrapped.asScala.collect { settings.config.getConfig("akka.actor.router.type-mapping").root.unwrapped.asScala.collect {
case (key, value: String) (key -> value) case (key, value: String) (key value)
}.toMap }.toMap
config.root.asScala flatMap { config.root.asScala flatMap {
@ -198,8 +198,8 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce
s"[${args(0)._1.getName}] and optional [${args(1)._1.getName}] parameter", cause) s"[${args(0)._1.getName}] and optional [${args(1)._1.getName}] parameter", cause)
// first try with Config param, and then with Config and DynamicAccess parameters // first try with Config param, and then with Config and DynamicAccess parameters
val args1 = List(classOf[Config] -> deployment2) val args1 = List(classOf[Config] deployment2)
val args2 = List(classOf[Config] -> deployment2, classOf[DynamicAccess] -> dynamicAccess) val args2 = List(classOf[Config] deployment2, classOf[DynamicAccess] dynamicAccess)
dynamicAccess.createInstanceFor[RouterConfig](fqn, args1).recover({ dynamicAccess.createInstanceFor[RouterConfig](fqn, args1).recover({
case e @ (_: IllegalArgumentException | _: ConfigException) throw e case e @ (_: IllegalArgumentException | _: ConfigException) throw e
case e: NoSuchMethodException case e: NoSuchMethodException

View file

@ -150,5 +150,5 @@ abstract class ExtensionKey[T <: Extension](implicit m: ClassTag[T]) extends Ext
def this(clazz: Class[T]) = this()(ClassTag(clazz)) def this(clazz: Class[T]) = this()(ClassTag(clazz))
override def lookup(): ExtensionId[T] = this override def lookup(): ExtensionId[T] = this
def createExtension(system: ExtendedActorSystem): T = system.dynamicAccess.createInstanceFor[T](m.runtimeClass, List(classOf[ExtendedActorSystem] -> system)).get def createExtension(system: ExtendedActorSystem): T = system.dynamicAccess.createInstanceFor[T](m.runtimeClass, List(classOf[ExtendedActorSystem] system)).get
} }

View file

@ -110,9 +110,10 @@ object FSM {
* This extractor is just convenience for matching a (S, S) pair, including a * This extractor is just convenience for matching a (S, S) pair, including a
* reminder what the new state is. * reminder what the new state is.
*/ */
object -> { object `->` {
def unapply[S](in: (S, S)) = Some(in) def unapply[S](in: (S, S)) = Some(in)
} }
val `→` = `->`
/** /**
* Log Entry of the [[akka.actor.LoggingFSM]], can be obtained by calling `getLog`. * Log Entry of the [[akka.actor.LoggingFSM]], can be obtained by calling `getLog`.
@ -319,7 +320,7 @@ trait FSM[S, D] extends Actor with Listeners with ActorLogging {
* This extractor is just convenience for matching a (S, S) pair, including a * This extractor is just convenience for matching a (S, S) pair, including a
* reminder what the new state is. * reminder what the new state is.
*/ */
val -> = FSM.-> val `->` = FSM.`->`
/** /**
* This case object is received in case of a state timeout. * This case object is received in case of a state timeout.

View file

@ -34,7 +34,8 @@ import akka.dispatch.AbstractNodeQueue
* scheduled possibly one tick later than they could be (if checking that * scheduled possibly one tick later than they could be (if checking that
* now() + delay &lt;= nextTick were done). * now() + delay &lt;= nextTick were done).
*/ */
class LightArrayRevolverScheduler(config: Config, class LightArrayRevolverScheduler(
config: Config,
log: LoggingAdapter, log: LoggingAdapter,
threadFactory: ThreadFactory) threadFactory: ThreadFactory)
extends Scheduler with Closeable { extends Scheduler with Closeable {
@ -88,7 +89,8 @@ class LightArrayRevolverScheduler(config: Config,
} }
} }
override def schedule(initialDelay: FiniteDuration, override def schedule(
initialDelay: FiniteDuration,
delay: FiniteDuration, delay: FiniteDuration,
runnable: Runnable)(implicit executor: ExecutionContext): Cancellable = { runnable: Runnable)(implicit executor: ExecutionContext): Cancellable = {
checkMaxDelay(roundUp(delay).toNanos) checkMaxDelay(roundUp(delay).toNanos)

View file

@ -176,7 +176,8 @@ private[akka] class RepointableActorRef(
protected def writeReplace(): AnyRef = SerializedActorRef(this) protected def writeReplace(): AnyRef = SerializedActorRef(this)
} }
private[akka] class UnstartedCell(val systemImpl: ActorSystemImpl, private[akka] class UnstartedCell(
val systemImpl: ActorSystemImpl,
val self: RepointableActorRef, val self: RepointableActorRef,
val props: Props, val props: Props,
val supervisor: InternalActorRef) extends Cell { val supervisor: InternalActorRef) extends Cell {

View file

@ -44,7 +44,8 @@ trait Scheduler {
initialDelay: FiniteDuration, initialDelay: FiniteDuration,
interval: FiniteDuration, interval: FiniteDuration,
receiver: ActorRef, receiver: ActorRef,
message: Any)(implicit executor: ExecutionContext, message: Any)(implicit
executor: ExecutionContext,
sender: ActorRef = Actor.noSender): Cancellable = sender: ActorRef = Actor.noSender): Cancellable =
schedule(initialDelay, interval, new Runnable { schedule(initialDelay, interval, new Runnable {
def run = { def run = {
@ -72,7 +73,8 @@ trait Scheduler {
final def schedule( final def schedule(
initialDelay: FiniteDuration, initialDelay: FiniteDuration,
interval: FiniteDuration)(f: Unit)( interval: FiniteDuration)(f: Unit)(
implicit executor: ExecutionContext): Cancellable = implicit
executor: ExecutionContext): Cancellable =
schedule(initialDelay, interval, new Runnable { override def run = f }) schedule(initialDelay, interval, new Runnable { override def run = f })
/** /**
@ -105,7 +107,8 @@ trait Scheduler {
final def scheduleOnce( final def scheduleOnce(
delay: FiniteDuration, delay: FiniteDuration,
receiver: ActorRef, receiver: ActorRef,
message: Any)(implicit executor: ExecutionContext, message: Any)(implicit
executor: ExecutionContext,
sender: ActorRef = Actor.noSender): Cancellable = sender: ActorRef = Actor.noSender): Cancellable =
scheduleOnce(delay, new Runnable { scheduleOnce(delay, new Runnable {
override def run = receiver ! message override def run = receiver ! message
@ -118,7 +121,8 @@ trait Scheduler {
* Scala API * Scala API
*/ */
final def scheduleOnce(delay: FiniteDuration)(f: Unit)( final def scheduleOnce(delay: FiniteDuration)(f: Unit)(
implicit executor: ExecutionContext): Cancellable = implicit
executor: ExecutionContext): Cancellable =
scheduleOnce(delay, new Runnable { override def run = f }) scheduleOnce(delay, new Runnable { override def run = f })
/** /**

View file

@ -536,7 +536,8 @@ final case class TypedProps[T <: AnyRef] protected[TypedProps] (
* appended in the sequence of interfaces. * appended in the sequence of interfaces.
*/ */
def this(implementation: Class[T]) = def this(implementation: Class[T]) =
this(interfaces = TypedProps.extractInterfaces(implementation), this(
interfaces = TypedProps.extractInterfaces(implementation),
creator = instantiator(implementation)) creator = instantiator(implementation))
/** /**
@ -546,7 +547,8 @@ final case class TypedProps[T <: AnyRef] protected[TypedProps] (
* appended in the sequence of interfaces. * appended in the sequence of interfaces.
*/ */
def this(interface: Class[_ >: T], implementation: Creator[T]) = def this(interface: Class[_ >: T], implementation: Creator[T]) =
this(interfaces = TypedProps.extractInterfaces(interface), this(
interfaces = TypedProps.extractInterfaces(interface),
creator = implementation.create _) creator = implementation.create _)
/** /**
@ -556,7 +558,8 @@ final case class TypedProps[T <: AnyRef] protected[TypedProps] (
* appended in the sequence of interfaces. * appended in the sequence of interfaces.
*/ */
def this(interface: Class[_ >: T], implementation: Class[T]) = def this(interface: Class[_ >: T], implementation: Class[T]) =
this(interfaces = TypedProps.extractInterfaces(interface), this(
interfaces = TypedProps.extractInterfaces(interface),
creator = instantiator(implementation)) creator = instantiator(implementation))
/** /**

View file

@ -62,7 +62,8 @@ private[akka] trait Dispatch { this: ActorCell ⇒
if (req isInstance mbox.messageQueue) Create(None) if (req isInstance mbox.messageQueue) Create(None)
else { else {
val gotType = if (mbox.messageQueue == null) "null" else mbox.messageQueue.getClass.getName val gotType = if (mbox.messageQueue == null) "null" else mbox.messageQueue.getClass.getName
Create(Some(ActorInitializationException(self, Create(Some(ActorInitializationException(
self,
s"Actor [$self] requires mailbox type [$req] got [$gotType]"))) s"Actor [$self] requires mailbox type [$req] got [$gotType]")))
} }
case _ Create(None) case _ Create(None)

View file

@ -324,8 +324,8 @@ abstract class MessageDispatcherConfigurator(_config: Config, val prerequisites:
case "thread-pool-executor" new ThreadPoolExecutorConfigurator(config.getConfig("thread-pool-executor"), prerequisites) case "thread-pool-executor" new ThreadPoolExecutorConfigurator(config.getConfig("thread-pool-executor"), prerequisites)
case fqcn case fqcn
val args = List( val args = List(
classOf[Config] -> config, classOf[Config] config,
classOf[DispatcherPrerequisites] -> prerequisites) classOf[DispatcherPrerequisites] prerequisites)
prerequisites.dynamicAccess.createInstanceFor[ExecutorServiceConfigurator](fqcn, args).recover({ prerequisites.dynamicAccess.createInstanceFor[ExecutorServiceConfigurator](fqcn, args).recover({
case exception throw new IllegalArgumentException( case exception throw new IllegalArgumentException(
("""Cannot instantiate ExecutorServiceConfigurator ("executor = [%s]"), defined in [%s], ("""Cannot instantiate ExecutorServiceConfigurator ("executor = [%s]"), defined in [%s],
@ -379,12 +379,14 @@ object ForkJoinExecutorConfigurator {
/** /**
* INTERNAL AKKA USAGE ONLY * INTERNAL AKKA USAGE ONLY
*/ */
final class AkkaForkJoinPool(parallelism: Int, final class AkkaForkJoinPool(
parallelism: Int,
threadFactory: ForkJoinPool.ForkJoinWorkerThreadFactory, threadFactory: ForkJoinPool.ForkJoinWorkerThreadFactory,
unhandledExceptionHandler: Thread.UncaughtExceptionHandler, unhandledExceptionHandler: Thread.UncaughtExceptionHandler,
asyncMode: Boolean) asyncMode: Boolean)
extends ForkJoinPool(parallelism, threadFactory, unhandledExceptionHandler, asyncMode) with LoadMetrics { extends ForkJoinPool(parallelism, threadFactory, unhandledExceptionHandler, asyncMode) with LoadMetrics {
def this(parallelism: Int, def this(
parallelism: Int,
threadFactory: ForkJoinPool.ForkJoinWorkerThreadFactory, threadFactory: ForkJoinPool.ForkJoinWorkerThreadFactory,
unhandledExceptionHandler: Thread.UncaughtExceptionHandler) = this(parallelism, threadFactory, unhandledExceptionHandler, asyncMode = true) unhandledExceptionHandler: Thread.UncaughtExceptionHandler) = this(parallelism, threadFactory, unhandledExceptionHandler, asyncMode = true)
@ -427,7 +429,8 @@ class ForkJoinExecutorConfigurator(config: Config, prerequisites: DispatcherPrer
case x throw new IllegalStateException("The prerequisites for the ForkJoinExecutorConfigurator is a ForkJoinPool.ForkJoinWorkerThreadFactory!") case x throw new IllegalStateException("The prerequisites for the ForkJoinExecutorConfigurator is a ForkJoinPool.ForkJoinWorkerThreadFactory!")
} }
class ForkJoinExecutorServiceFactory(val threadFactory: ForkJoinPool.ForkJoinWorkerThreadFactory, class ForkJoinExecutorServiceFactory(
val threadFactory: ForkJoinPool.ForkJoinWorkerThreadFactory,
val parallelism: Int, val parallelism: Int,
val asyncMode: Boolean) extends ExecutorServiceFactory { val asyncMode: Boolean) extends ExecutorServiceFactory {
def this(threadFactory: ForkJoinPool.ForkJoinWorkerThreadFactory, parallelism: Int) = this(threadFactory, parallelism, asyncMode = true) def this(threadFactory: ForkJoinPool.ForkJoinWorkerThreadFactory, parallelism: Int) = this(threadFactory, parallelism, asyncMode = true)

View file

@ -135,13 +135,13 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc
def simpleName = id.substring(id.lastIndexOf('.') + 1) def simpleName = id.substring(id.lastIndexOf('.') + 1)
idConfig(id) idConfig(id)
.withFallback(appConfig) .withFallback(appConfig)
.withFallback(ConfigFactory.parseMap(Map("name" -> simpleName).asJava)) .withFallback(ConfigFactory.parseMap(Map("name" simpleName).asJava))
.withFallback(defaultDispatcherConfig) .withFallback(defaultDispatcherConfig)
} }
private def idConfig(id: String): Config = { private def idConfig(id: String): Config = {
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
ConfigFactory.parseMap(Map("id" -> id).asJava) ConfigFactory.parseMap(Map("id" id).asJava)
} }
/** /**
@ -180,7 +180,7 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc
classOf[BalancingDispatcherConfigurator].getName) classOf[BalancingDispatcherConfigurator].getName)
case "PinnedDispatcher" new PinnedDispatcherConfigurator(cfg, prerequisites) case "PinnedDispatcher" new PinnedDispatcherConfigurator(cfg, prerequisites)
case fqn case fqn
val args = List(classOf[Config] -> cfg, classOf[DispatcherPrerequisites] -> prerequisites) val args = List(classOf[Config] cfg, classOf[DispatcherPrerequisites] prerequisites)
prerequisites.dynamicAccess.createInstanceFor[MessageDispatcherConfigurator](fqn, args).recover({ prerequisites.dynamicAccess.createInstanceFor[MessageDispatcherConfigurator](fqn, args).recover({
case exception case exception
throw new ConfigurationException( throw new ConfigurationException(
@ -288,7 +288,8 @@ class PinnedDispatcherConfigurator(config: Config, prerequisites: DispatcherPrer
case e: ThreadPoolExecutorConfigurator e.threadPoolConfig case e: ThreadPoolExecutorConfigurator e.threadPoolConfig
case other case other
prerequisites.eventStream.publish( prerequisites.eventStream.publish(
Warning("PinnedDispatcherConfigurator", Warning(
"PinnedDispatcherConfigurator",
this.getClass, this.getClass,
"PinnedDispatcher [%s] not configured to use ThreadPoolExecutor, falling back to default config.".format( "PinnedDispatcher [%s] not configured to use ThreadPoolExecutor, falling back to default config.".format(
config.getString("id")))) config.getString("id"))))

View file

@ -657,7 +657,8 @@ final case class BoundedMailbox(val capacity: Int, override val pushTimeOut: Fin
extends MailboxType with ProducesMessageQueue[BoundedMailbox.MessageQueue] extends MailboxType with ProducesMessageQueue[BoundedMailbox.MessageQueue]
with ProducesPushTimeoutSemanticsMailbox { with ProducesPushTimeoutSemanticsMailbox {
def this(settings: ActorSystem.Settings, config: Config) = this(config.getInt("mailbox-capacity"), def this(settings: ActorSystem.Settings, config: Config) = this(
config.getInt("mailbox-capacity"),
config.getNanosDuration("mailbox-push-timeout-time")) config.getNanosDuration("mailbox-push-timeout-time"))
if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedMailbox can not be negative") if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedMailbox can not be negative")
@ -782,7 +783,8 @@ case class BoundedDequeBasedMailbox( final val capacity: Int, override final val
extends MailboxType with ProducesMessageQueue[BoundedDequeBasedMailbox.MessageQueue] extends MailboxType with ProducesMessageQueue[BoundedDequeBasedMailbox.MessageQueue]
with ProducesPushTimeoutSemanticsMailbox { with ProducesPushTimeoutSemanticsMailbox {
def this(settings: ActorSystem.Settings, config: Config) = this(config.getInt("mailbox-capacity"), def this(settings: ActorSystem.Settings, config: Config) = this(
config.getInt("mailbox-capacity"),
config.getNanosDuration("mailbox-push-timeout-time")) config.getNanosDuration("mailbox-push-timeout-time"))
if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedDequeBasedMailbox can not be negative") if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedDequeBasedMailbox can not be negative")
@ -858,7 +860,8 @@ object UnboundedControlAwareMailbox {
final case class BoundedControlAwareMailbox(capacity: Int, override final val pushTimeOut: FiniteDuration) extends MailboxType final case class BoundedControlAwareMailbox(capacity: Int, override final val pushTimeOut: FiniteDuration) extends MailboxType
with ProducesMessageQueue[BoundedControlAwareMailbox.MessageQueue] with ProducesMessageQueue[BoundedControlAwareMailbox.MessageQueue]
with ProducesPushTimeoutSemanticsMailbox { with ProducesPushTimeoutSemanticsMailbox {
def this(settings: ActorSystem.Settings, config: Config) = this(config.getInt("mailbox-capacity"), def this(settings: ActorSystem.Settings, config: Config) = this(
config.getInt("mailbox-capacity"),
config.getNanosDuration("mailbox-push-timeout-time")) config.getNanosDuration("mailbox-push-timeout-time"))
def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue = new BoundedControlAwareMailbox.MessageQueue(capacity, pushTimeOut) def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue = new BoundedControlAwareMailbox.MessageQueue(capacity, pushTimeOut)

View file

@ -187,7 +187,7 @@ private[akka] class Mailboxes(
val mailboxType = conf.getString("mailbox-type") match { val mailboxType = conf.getString("mailbox-type") match {
case "" throw new ConfigurationException(s"The setting mailbox-type, defined in [$id] is empty") case "" throw new ConfigurationException(s"The setting mailbox-type, defined in [$id] is empty")
case fqcn case fqcn
val args = List(classOf[ActorSystem.Settings] -> settings, classOf[Config] -> conf) val args = List(classOf[ActorSystem.Settings] settings, classOf[Config] conf)
dynamicAccess.createInstanceFor[MailboxType](fqcn, args).recover({ dynamicAccess.createInstanceFor[MailboxType](fqcn, args).recover({
case exception case exception
throw new IllegalArgumentException( throw new IllegalArgumentException(
@ -228,7 +228,7 @@ private[akka] class Mailboxes(
//INTERNAL API //INTERNAL API
private def config(id: String): Config = { private def config(id: String): Config = {
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
ConfigFactory.parseMap(Map("id" -> id).asJava) ConfigFactory.parseMap(Map("id" id).asJava)
.withFallback(settings.config.getConfig(id)) .withFallback(settings.config.getConfig(id))
.withFallback(defaultMailboxConfig) .withFallback(defaultMailboxConfig)
} }

View file

@ -20,7 +20,8 @@ class PinnedDispatcher(
_id: String, _id: String,
_shutdownTimeout: FiniteDuration, _shutdownTimeout: FiniteDuration,
_threadPoolConfig: ThreadPoolConfig) _threadPoolConfig: ThreadPoolConfig)
extends Dispatcher(_configurator, extends Dispatcher(
_configurator,
_id, _id,
Int.MaxValue, Int.MaxValue,
Duration.Zero, Duration.Zero,

View file

@ -65,7 +65,8 @@ trait ExecutorServiceFactoryProvider {
/** /**
* A small configuration DSL to create ThreadPoolExecutors that can be provided as an ExecutorServiceFactoryProvider to Dispatcher * A small configuration DSL to create ThreadPoolExecutors that can be provided as an ExecutorServiceFactoryProvider to Dispatcher
*/ */
final case class ThreadPoolConfig(allowCorePoolTimeout: Boolean = ThreadPoolConfig.defaultAllowCoreThreadTimeout, final case class ThreadPoolConfig(
allowCorePoolTimeout: Boolean = ThreadPoolConfig.defaultAllowCoreThreadTimeout,
corePoolSize: Int = ThreadPoolConfig.defaultCorePoolSize, corePoolSize: Int = ThreadPoolConfig.defaultCorePoolSize,
maxPoolSize: Int = ThreadPoolConfig.defaultMaxPoolSize, maxPoolSize: Int = ThreadPoolConfig.defaultMaxPoolSize,
threadTimeout: Duration = ThreadPoolConfig.defaultTimeout, threadTimeout: Duration = ThreadPoolConfig.defaultTimeout,
@ -173,7 +174,8 @@ object MonitorableThreadFactory {
} }
} }
final case class MonitorableThreadFactory(name: String, final case class MonitorableThreadFactory(
name: String,
daemonic: Boolean, daemonic: Boolean,
contextClassLoader: Option[ClassLoader], contextClassLoader: Option[ClassLoader],
exceptionHandler: Thread.UncaughtExceptionHandler = MonitorableThreadFactory.doNothing, exceptionHandler: Thread.UncaughtExceptionHandler = MonitorableThreadFactory.doNothing,

View file

@ -5,6 +5,7 @@
package akka.io package akka.io
import java.nio.ByteBuffer import java.nio.ByteBuffer
import scala.util.control.NonFatal
trait BufferPool { trait BufferPool {
def acquire(): ByteBuffer def acquire(): ByteBuffer
@ -54,11 +55,42 @@ private[akka] class DirectByteBufferPool(defaultBufferSize: Int, maxPoolEntries:
} }
} }
private final def offerBufferToPool(buf: ByteBuffer): Unit = private final def offerBufferToPool(buf: ByteBuffer): Unit = {
val clean =
pool.synchronized { pool.synchronized {
if (buffersInPool < maxPoolEntries) { if (buffersInPool < maxPoolEntries) {
pool(buffersInPool) = buf pool(buffersInPool) = buf
buffersInPool += 1 buffersInPool += 1
} // else let the buffer be gc'd false
} else {
// try to clean it outside the lock, or let the buffer be gc'd
true
}
}
if (clean)
tryCleanDirectByteBuffer(buf)
}
/**
* DirectByteBuffers are garbage collected by using a phantom reference and a
* reference queue. Every once a while, the JVM checks the reference queue and
* cleans the DirectByteBuffers. However, as this doesn't happen
* immediately after discarding all references to a DirectByteBuffer, it's
* easy to OutOfMemoryError yourself using DirectByteBuffers. This function
* explicitly calls the Cleaner method of a DirectByteBuffer.
*
* Utilizes reflection to avoid dependency to `sun.misc.Cleaner`.
*/
private final def tryCleanDirectByteBuffer(toBeDestroyed: ByteBuffer): Unit = try {
if (toBeDestroyed.isDirect) {
val cleanerMethod = toBeDestroyed.getClass().getMethod("cleaner")
cleanerMethod.setAccessible(true)
val cleaner = cleanerMethod.invoke(toBeDestroyed)
val cleanMethod = cleaner.getClass().getMethod("clean")
cleanMethod.setAccessible(true)
cleanMethod.invoke(cleaner)
}
} catch {
case NonFatal(_) // attempt failed, ok
} }
} }

View file

@ -58,7 +58,7 @@ object SimpleDnsCache {
new Cache( new Cache(
queue + new ExpiryEntry(answer.name, until), queue + new ExpiryEntry(answer.name, until),
cache + (answer.name -> CacheEntry(answer, until)), cache + (answer.name CacheEntry(answer, until)),
clock) clock)
} }

View file

@ -110,7 +110,8 @@ object Tcp extends ExtensionId[TcpExt] with ExtensionIdProvider {
* @param localAddress optionally specifies a specific address to bind to * @param localAddress optionally specifies a specific address to bind to
* @param options Please refer to the `Tcp.SO` object for a list of all supported options. * @param options Please refer to the `Tcp.SO` object for a list of all supported options.
*/ */
final case class Connect(remoteAddress: InetSocketAddress, final case class Connect(
remoteAddress: InetSocketAddress,
localAddress: Option[InetSocketAddress] = None, localAddress: Option[InetSocketAddress] = None,
options: immutable.Traversable[SocketOption] = Nil, options: immutable.Traversable[SocketOption] = Nil,
timeout: Option[FiniteDuration] = None, timeout: Option[FiniteDuration] = None,
@ -135,7 +136,8 @@ object Tcp extends ExtensionId[TcpExt] with ExtensionIdProvider {
* *
* @param options Please refer to the `Tcp.SO` object for a list of all supported options. * @param options Please refer to the `Tcp.SO` object for a list of all supported options.
*/ */
final case class Bind(handler: ActorRef, final case class Bind(
handler: ActorRef,
localAddress: InetSocketAddress, localAddress: InetSocketAddress,
backlog: Int = 100, backlog: Int = 100,
options: immutable.Traversable[SocketOption] = Nil, options: immutable.Traversable[SocketOption] = Nil,
@ -624,7 +626,8 @@ object TcpMessage {
* @param timeout is the desired connection timeout, `null` means "no timeout" * @param timeout is the desired connection timeout, `null` means "no timeout"
* @param pullMode enables pull based reading from the connection * @param pullMode enables pull based reading from the connection
*/ */
def connect(remoteAddress: InetSocketAddress, def connect(
remoteAddress: InetSocketAddress,
localAddress: InetSocketAddress, localAddress: InetSocketAddress,
options: JIterable[SocketOption], options: JIterable[SocketOption],
timeout: FiniteDuration, timeout: FiniteDuration,
@ -658,7 +661,8 @@ object TcpMessage {
* @param pullMode enables pull based accepting and of connections and pull * @param pullMode enables pull based accepting and of connections and pull
* based reading from the accepted connections. * based reading from the accepted connections.
*/ */
def bind(handler: ActorRef, def bind(
handler: ActorRef,
endpoint: InetSocketAddress, endpoint: InetSocketAddress,
backlog: Int, backlog: Int,
options: JIterable[SocketOption], options: JIterable[SocketOption],
@ -666,7 +670,8 @@ object TcpMessage {
/** /**
* Open a listening socket without specifying options. * Open a listening socket without specifying options.
*/ */
def bind(handler: ActorRef, def bind(
handler: ActorRef,
endpoint: InetSocketAddress, endpoint: InetSocketAddress,
backlog: Int): Command = Bind(handler, endpoint, backlog, Nil) backlog: Int): Command = Bind(handler, endpoint, backlog, Nil)

View file

@ -479,7 +479,8 @@ private[io] object TcpConnection {
/** /**
* Groups required connection-related data that are only available once the connection has been fully established. * Groups required connection-related data that are only available once the connection has been fully established.
*/ */
final case class ConnectionInfo(registration: ChannelRegistration, final case class ConnectionInfo(
registration: ChannelRegistration,
handler: ActorRef, handler: ActorRef,
keepOpenOnPeerClosed: Boolean, keepOpenOnPeerClosed: Boolean,
useResumeWriting: Boolean) useResumeWriting: Boolean)

View file

@ -15,7 +15,8 @@ import akka.io.Inet.SocketOption
* *
* INTERNAL API * INTERNAL API
*/ */
private[io] class TcpIncomingConnection(_tcp: TcpExt, private[io] class TcpIncomingConnection(
_tcp: TcpExt,
_channel: SocketChannel, _channel: SocketChannel,
registry: ChannelRegistry, registry: ChannelRegistry,
bindHandler: ActorRef, bindHandler: ActorRef,

View file

@ -31,7 +31,8 @@ private[io] object TcpListener {
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[io] class TcpListener(selectorRouter: ActorRef, private[io] class TcpListener(
selectorRouter: ActorRef,
tcp: TcpExt, tcp: TcpExt,
channelRegistry: ChannelRegistry, channelRegistry: ChannelRegistry,
bindCommander: ActorRef, bindCommander: ActorRef,

View file

@ -19,7 +19,8 @@ import akka.io.Tcp._
* *
* INTERNAL API * INTERNAL API
*/ */
private[io] class TcpOutgoingConnection(_tcp: TcpExt, private[io] class TcpOutgoingConnection(
_tcp: TcpExt,
channelRegistry: ChannelRegistry, channelRegistry: ChannelRegistry,
commander: ActorRef, commander: ActorRef,
connect: Connect) connect: Connect)

View file

@ -92,7 +92,8 @@ object Udp extends ExtensionId[UdpExt] with ExtensionIdProvider {
* The listener actor for the newly bound port will reply with a [[Bound]] * The listener actor for the newly bound port will reply with a [[Bound]]
* message, or the manager will reply with a [[CommandFailed]] message. * message, or the manager will reply with a [[CommandFailed]] message.
*/ */
final case class Bind(handler: ActorRef, final case class Bind(
handler: ActorRef,
localAddress: InetSocketAddress, localAddress: InetSocketAddress,
options: immutable.Traversable[SocketOption] = Nil) extends Command options: immutable.Traversable[SocketOption] = Nil) extends Command

View file

@ -84,7 +84,8 @@ object UdpConnected extends ExtensionId[UdpConnectedExt] with ExtensionIdProvide
* which is restricted to sending to and receiving from the given `remoteAddress`. * which is restricted to sending to and receiving from the given `remoteAddress`.
* All received datagrams will be sent to the designated `handler` actor. * All received datagrams will be sent to the designated `handler` actor.
*/ */
final case class Connect(handler: ActorRef, final case class Connect(
handler: ActorRef,
remoteAddress: InetSocketAddress, remoteAddress: InetSocketAddress,
localAddress: Option[InetSocketAddress] = None, localAddress: Option[InetSocketAddress] = None,
options: immutable.Traversable[SocketOption] = Nil) extends Command options: immutable.Traversable[SocketOption] = Nil) extends Command
@ -176,20 +177,23 @@ object UdpConnectedMessage {
* which is restricted to sending to and receiving from the given `remoteAddress`. * which is restricted to sending to and receiving from the given `remoteAddress`.
* All received datagrams will be sent to the designated `handler` actor. * All received datagrams will be sent to the designated `handler` actor.
*/ */
def connect(handler: ActorRef, def connect(
handler: ActorRef,
remoteAddress: InetSocketAddress, remoteAddress: InetSocketAddress,
localAddress: InetSocketAddress, localAddress: InetSocketAddress,
options: JIterable[SocketOption]): Command = Connect(handler, remoteAddress, Some(localAddress), options) options: JIterable[SocketOption]): Command = Connect(handler, remoteAddress, Some(localAddress), options)
/** /**
* Connect without specifying the `localAddress`. * Connect without specifying the `localAddress`.
*/ */
def connect(handler: ActorRef, def connect(
handler: ActorRef,
remoteAddress: InetSocketAddress, remoteAddress: InetSocketAddress,
options: JIterable[SocketOption]): Command = Connect(handler, remoteAddress, None, options) options: JIterable[SocketOption]): Command = Connect(handler, remoteAddress, None, options)
/** /**
* Connect without specifying the `localAddress` or `options`. * Connect without specifying the `localAddress` or `options`.
*/ */
def connect(handler: ActorRef, def connect(
handler: ActorRef,
remoteAddress: InetSocketAddress): Command = Connect(handler, remoteAddress, None, Nil) remoteAddress: InetSocketAddress): Command = Connect(handler, remoteAddress, None, Nil)
/** /**

View file

@ -18,7 +18,8 @@ import akka.io.UdpConnected._
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[io] class UdpConnection(udpConn: UdpConnectedExt, private[io] class UdpConnection(
udpConn: UdpConnectedExt,
channelRegistry: ChannelRegistry, channelRegistry: ChannelRegistry,
commander: ActorRef, commander: ActorRef,
connect: Connect) connect: Connect)
@ -153,7 +154,8 @@ private[io] class UdpConnection(udpConn: UdpConnectedExt,
thunk thunk
} catch { } catch {
case NonFatal(e) case NonFatal(e)
log.debug("Failure while connecting UDP channel to remote address [{}] local address [{}]: {}", log.debug(
"Failure while connecting UDP channel to remote address [{}] local address [{}]: {}",
remoteAddress, localAddress.getOrElse("undefined"), e) remoteAddress, localAddress.getOrElse("undefined"), e)
commander ! CommandFailed(connect) commander ! CommandFailed(connect)
context.stop(self) context.stop(self)

View file

@ -19,7 +19,8 @@ import akka.io.Udp._
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[io] class UdpListener(val udp: UdpExt, private[io] class UdpListener(
val udp: UdpExt,
channelRegistry: ChannelRegistry, channelRegistry: ChannelRegistry,
bindCommander: ActorRef, bindCommander: ActorRef,
bind: Bind) bind: Bind)

View file

@ -14,7 +14,8 @@ import akka.actor._
/** /**
* INTERNAL API * INTERNAL API
*/ */
private[io] class UdpSender(val udp: UdpExt, private[io] class UdpSender(
val udp: UdpExt,
channelRegistry: ChannelRegistry, channelRegistry: ChannelRegistry,
commander: ActorRef, commander: ActorRef,
options: immutable.Traversable[SocketOption]) options: immutable.Traversable[SocketOption])

View file

@ -51,7 +51,8 @@ private[io] trait WithUdpSend {
} catch { } catch {
case NonFatal(e) case NonFatal(e)
sender() ! CommandFailed(send) sender() ! CommandFailed(send)
log.debug("Failure while sending UDP datagram to remote address [{}]: {}", log.debug(
"Failure while sending UDP datagram to remote address [{}]: {}",
send.target, e) send.target, e)
retriedSend = false retriedSend = false
pendingSend = null pendingSend = null

View file

@ -112,12 +112,14 @@ final case class BalancingPool(
// dispatcher of this pool // dispatcher of this pool
val deployDispatcherConfigPath = s"akka.actor.deployment.$deployPath.pool-dispatcher" val deployDispatcherConfigPath = s"akka.actor.deployment.$deployPath.pool-dispatcher"
val systemConfig = context.system.settings.config val systemConfig = context.system.settings.config
val dispatcherConfig = context.system.dispatchers.config(dispatcherId, val dispatcherConfig = context.system.dispatchers.config(
dispatcherId,
// use the user defined 'pool-dispatcher' config as fallback, if any // use the user defined 'pool-dispatcher' config as fallback, if any
if (systemConfig.hasPath(deployDispatcherConfigPath)) systemConfig.getConfig(deployDispatcherConfigPath) if (systemConfig.hasPath(deployDispatcherConfigPath)) systemConfig.getConfig(deployDispatcherConfigPath)
else ConfigFactory.empty) else ConfigFactory.empty)
dispatchers.registerConfigurator(dispatcherId, new BalancingDispatcherConfigurator(dispatcherConfig, dispatchers.registerConfigurator(dispatcherId, new BalancingDispatcherConfigurator(
dispatcherConfig,
dispatchers.prerequisites)) dispatchers.prerequisites))
} }

View file

@ -39,7 +39,8 @@ class ConsistentHash[T: ClassTag] private (nodes: immutable.SortedMap[Int, T], v
*/ */
def :+(node: T): ConsistentHash[T] = { def :+(node: T): ConsistentHash[T] = {
val nodeHash = hashFor(node.toString) val nodeHash = hashFor(node.toString)
new ConsistentHash(nodes ++ ((1 to virtualNodesFactor) map { r (concatenateNodeHash(nodeHash, r) -> node) }), new ConsistentHash(
nodes ++ ((1 to virtualNodesFactor) map { r (concatenateNodeHash(nodeHash, r) node) }),
virtualNodesFactor) virtualNodesFactor)
} }
@ -57,7 +58,8 @@ class ConsistentHash[T: ClassTag] private (nodes: immutable.SortedMap[Int, T], v
*/ */
def :-(node: T): ConsistentHash[T] = { def :-(node: T): ConsistentHash[T] = {
val nodeHash = hashFor(node.toString) val nodeHash = hashFor(node.toString)
new ConsistentHash(nodes -- ((1 to virtualNodesFactor) map { r concatenateNodeHash(nodeHash, r) }), new ConsistentHash(
nodes -- ((1 to virtualNodesFactor) map { r concatenateNodeHash(nodeHash, r) }),
virtualNodesFactor) virtualNodesFactor)
} }
@ -110,12 +112,13 @@ class ConsistentHash[T: ClassTag] private (nodes: immutable.SortedMap[Int, T], v
object ConsistentHash { object ConsistentHash {
def apply[T: ClassTag](nodes: Iterable[T], virtualNodesFactor: Int): ConsistentHash[T] = { def apply[T: ClassTag](nodes: Iterable[T], virtualNodesFactor: Int): ConsistentHash[T] = {
new ConsistentHash(immutable.SortedMap.empty[Int, T] ++ new ConsistentHash(
immutable.SortedMap.empty[Int, T] ++
(for { (for {
node nodes node nodes
nodeHash = hashFor(node.toString) nodeHash = hashFor(node.toString)
vnode 1 to virtualNodesFactor vnode 1 to virtualNodesFactor
} yield (concatenateNodeHash(nodeHash, vnode) -> node)), } yield (concatenateNodeHash(nodeHash, vnode) node)),
virtualNodesFactor) virtualNodesFactor)
} }

View file

@ -219,7 +219,8 @@ final case class ConsistentHashingRoutingLogic(
case _ if hashMapping.isDefinedAt(message) target(hashMapping(message)) case _ if hashMapping.isDefinedAt(message) target(hashMapping(message))
case hashable: ConsistentHashable target(hashable.consistentHashKey) case hashable: ConsistentHashable target(hashable.consistentHashKey)
case other case other
log.warning("Message [{}] must be handled by hashMapping, or implement [{}] or be wrapped in [{}]", log.warning(
"Message [{}] must be handled by hashMapping, or implement [{}] or be wrapped in [{}]",
message.getClass.getName, classOf[ConsistentHashable].getName, message.getClass.getName, classOf[ConsistentHashable].getName,
classOf[ConsistentHashableEnvelope].getName) classOf[ConsistentHashableEnvelope].getName)
NoRoutee NoRoutee

View file

@ -72,7 +72,8 @@ final case class RoundRobinPool(
extends Pool with PoolOverrideUnsetConfig[RoundRobinPool] { extends Pool with PoolOverrideUnsetConfig[RoundRobinPool] {
def this(config: Config) = def this(config: Config) =
this(nrOfInstances = config.getInt("nr-of-instances"), this(
nrOfInstances = config.getInt("nr-of-instances"),
resizer = Resizer.fromConfig(config), resizer = Resizer.fromConfig(config),
usePoolDispatcher = config.hasPath("pool-dispatcher")) usePoolDispatcher = config.hasPath("pool-dispatcher"))

View file

@ -154,7 +154,8 @@ private[akka] class RouterActor extends Actor {
} }
val routingLogicController: Option[ActorRef] = cell.routerConfig.routingLogicController( val routingLogicController: Option[ActorRef] = cell.routerConfig.routingLogicController(
cell.router.logic).map(props context.actorOf(props.withDispatcher(context.props.dispatcher), cell.router.logic).map(props context.actorOf(
props.withDispatcher(context.props.dispatcher),
name = "routingLogicController")) name = "routingLogicController"))
def receive = { def receive = {

View file

@ -3,7 +3,6 @@
*/ */
package akka.routing package akka.routing
import scala.collection.immutable import scala.collection.immutable
import akka.ConfigurationException import akka.ConfigurationException
import akka.actor.ActorContext import akka.actor.ActorContext
@ -297,7 +296,8 @@ case object FromConfig extends FromConfig {
* (defaults to default-dispatcher). * (defaults to default-dispatcher).
*/ */
@SerialVersionUID(1L) @SerialVersionUID(1L)
class FromConfig(override val resizer: Option[Resizer], class FromConfig(
override val resizer: Option[Resizer],
override val supervisorStrategy: SupervisorStrategy, override val supervisorStrategy: SupervisorStrategy,
override val routerDispatcher: String) extends Pool { override val routerDispatcher: String) extends Pool {

View file

@ -45,7 +45,8 @@ class SmallestMailboxRoutingLogic extends RoutingLogic {
// 4. An ActorRef with unknown mailbox size that isn't processing anything // 4. An ActorRef with unknown mailbox size that isn't processing anything
// 5. An ActorRef with a known mailbox size // 5. An ActorRef with a known mailbox size
// 6. An ActorRef without any messages // 6. An ActorRef without any messages
@tailrec private def selectNext(targets: immutable.IndexedSeq[Routee], @tailrec private def selectNext(
targets: immutable.IndexedSeq[Routee],
proposedTarget: Routee = NoRoutee, proposedTarget: Routee = NoRoutee,
currentScore: Long = Long.MaxValue, currentScore: Long = Long.MaxValue,
at: Int = 0, at: Int = 0,

View file

@ -38,7 +38,7 @@ object Serialization {
private final def configToMap(path: String): Map[String, String] = { private final def configToMap(path: String): Map[String, String] = {
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
config.getConfig(path).root.unwrapped.asScala.toMap map { case (k, v) (k -> v.toString) } config.getConfig(path).root.unwrapped.asScala.toMap map { case (k, v) (k v.toString) }
} }
} }
@ -236,7 +236,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
* loading is performed by the systems [[akka.actor.DynamicAccess]]. * loading is performed by the systems [[akka.actor.DynamicAccess]].
*/ */
def serializerOf(serializerFQN: String): Try[Serializer] = def serializerOf(serializerFQN: String): Try[Serializer] =
system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, List(classOf[ExtendedActorSystem] -> system)) recoverWith { system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, List(classOf[ExtendedActorSystem] system)) recoverWith {
case _: NoSuchMethodException system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, Nil) case _: NoSuchMethodException system.dynamicAccess.createInstanceFor[Serializer](serializerFQN, Nil)
} }
@ -245,7 +245,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
* By default always contains the following mapping: "java" -> akka.serialization.JavaSerializer * By default always contains the following mapping: "java" -> akka.serialization.JavaSerializer
*/ */
private val serializers: Map[String, Serializer] = private val serializers: Map[String, Serializer] =
for ((k: String, v: String) settings.Serializers) yield k -> serializerOf(v).get for ((k: String, v: String) settings.Serializers) yield k serializerOf(v).get
/** /**
* bindings is a Seq of tuple representing the mapping from Class to Serializer. * bindings is a Seq of tuple representing the mapping from Class to Serializer.
@ -286,7 +286,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
* Maps from a Serializer Identity (Int) to a Serializer instance (optimization) * Maps from a Serializer Identity (Int) to a Serializer instance (optimization)
*/ */
val serializerByIdentity: Map[Int, Serializer] = val serializerByIdentity: Map[Int, Serializer] =
Map(NullSerializer.identifier -> NullSerializer) ++ serializers map { case (_, v) (v.identifier, v) } Map(NullSerializer.identifier NullSerializer) ++ serializers map { case (_, v) (v.identifier, v) }
private val isJavaSerializationWarningEnabled = settings.config.getBoolean("akka.actor.warn-about-java-serializer-usage") private val isJavaSerializationWarningEnabled = settings.config.getBoolean("akka.actor.warn-about-java-serializer-usage")

View file

@ -7,15 +7,15 @@ object BoxedType {
import java.{ lang jl } import java.{ lang jl }
private val toBoxed = Map[Class[_], Class[_]]( private val toBoxed = Map[Class[_], Class[_]](
classOf[Boolean] -> classOf[jl.Boolean], classOf[Boolean] classOf[jl.Boolean],
classOf[Byte] -> classOf[jl.Byte], classOf[Byte] classOf[jl.Byte],
classOf[Char] -> classOf[jl.Character], classOf[Char] classOf[jl.Character],
classOf[Short] -> classOf[jl.Short], classOf[Short] classOf[jl.Short],
classOf[Int] -> classOf[jl.Integer], classOf[Int] classOf[jl.Integer],
classOf[Long] -> classOf[jl.Long], classOf[Long] classOf[jl.Long],
classOf[Float] -> classOf[jl.Float], classOf[Float] classOf[jl.Float],
classOf[Double] -> classOf[jl.Double], classOf[Double] classOf[jl.Double],
classOf[Unit] -> classOf[scala.runtime.BoxedUnit]) classOf[Unit] classOf[scala.runtime.BoxedUnit])
final def apply(c: Class[_]): Class[_] = if (c.isPrimitive) toBoxed(c) else c final def apply(c: Class[_]): Class[_] = if (c.isPrimitive) toBoxed(c) else c
} }

View file

@ -357,7 +357,7 @@ object ByteString {
private[akka] object Companion { private[akka] object Companion {
private val companionMap = Seq(ByteString1, ByteString1C, ByteStrings). private val companionMap = Seq(ByteString1, ByteString1C, ByteStrings).
map(x x.SerializationIdentity -> x).toMap. map(x x.SerializationIdentity x).toMap.
withDefault(x throw new IllegalArgumentException("Invalid serialization id " + x)) withDefault(x throw new IllegalArgumentException("Invalid serialization id " + x))
def apply(from: Byte): Companion = companionMap(from) def apply(from: Byte): Companion = companionMap(from)

View file

@ -187,7 +187,7 @@ object LineNumbers {
val cl = c.getClassLoader val cl = c.getClassLoader
val r = cl.getResourceAsStream(resource) val r = cl.getResourceAsStream(resource)
if (debug) println(s"LNB: resource '$resource' resolved to stream $r") if (debug) println(s"LNB: resource '$resource' resolved to stream $r")
Option(r).map(_ -> None) Option(r).map(_ None)
} }
private def getStreamForLambda(l: AnyRef): Option[(InputStream, Some[String])] = private def getStreamForLambda(l: AnyRef): Option[(InputStream, Some[String])] =
@ -269,7 +269,7 @@ object LineNumbers {
val count = d.readUnsignedShort() val count = d.readUnsignedShort()
if (debug) println(s"LNB: reading $count methods") if (debug) println(s"LNB: reading $count methods")
if (c.contains("Code") && c.contains("LineNumberTable")) { if (c.contains("Code") && c.contains("LineNumberTable")) {
(1 to count).map(_ readMethod(d, c("Code"), c("LineNumberTable"), filter)).flatten.foldLeft(Int.MaxValue -> 0) { (1 to count).map(_ readMethod(d, c("Code"), c("LineNumberTable"), filter)).flatten.foldLeft(Int.MaxValue 0) {
case ((low, high), (start, end)) (Math.min(low, start), Math.max(high, end)) case ((low, high), (start, end)) (Math.min(low, start), Math.max(high, end))
} match { } match {
case (Int.MaxValue, 0) None case (Int.MaxValue, 0) None
@ -282,7 +282,8 @@ object LineNumbers {
} }
} }
private def readMethod(d: DataInputStream, private def readMethod(
d: DataInputStream,
codeTag: Int, codeTag: Int,
lineNumberTableTag: Int, lineNumberTableTag: Int,
filter: Option[String])(implicit c: Constants): Option[(Int, Int)] = { filter: Option[String])(implicit c: Constants): Option[(Int, Int)] = {
@ -315,7 +316,7 @@ object LineNumbers {
skip(d, 2) // start PC skip(d, 2) // start PC
d.readUnsignedShort() // finally: the line number d.readUnsignedShort() // finally: the line number
} }
Some(lines.min -> lines.max) Some(lines.min lines.max)
} }
} }
if (debug) println(s"LNB: nested attributes yielded: $possibleLines") if (debug) println(s"LNB: nested attributes yielded: $possibleLines")

View file

@ -127,7 +127,7 @@ private[akka] class SubclassifiedIndex[K, V] private (protected var values: Set[
if (!found) { if (!found) {
val v = values + value val v = values + value
val n = new Nonroot(root, key, v) val n = new Nonroot(root, key, v)
integrate(n) ++ n.innerAddValue(key, value) :+ (key -> v) integrate(n) ++ n.innerAddValue(key, value) :+ (key v)
} else ch } else ch
} }

View file

@ -44,7 +44,8 @@ class ForkJoinActorBenchmark {
| } | }
| } | }
| } | }
""".stripMargin)) """.stripMargin
))
} }
@TearDown(Level.Trial) @TearDown(Level.Trial)

View file

@ -46,7 +46,8 @@ class TellOnlyBenchmark {
| type = "akka.actor.TellOnlyBenchmark$$DroppingDispatcherConfigurator" | type = "akka.actor.TellOnlyBenchmark$$DroppingDispatcherConfigurator"
| mailbox-type = "akka.actor.TellOnlyBenchmark$$UnboundedDroppingMailbox" | mailbox-type = "akka.actor.TellOnlyBenchmark$$UnboundedDroppingMailbox"
| } | }
| """.stripMargin)) | """.stripMargin
))
} }
@TearDown(Level.Trial) @TearDown(Level.Trial)
@ -125,7 +126,8 @@ object TellOnlyBenchmark {
_throughput: Int, _throughput: Int,
_throughputDeadlineTime: Duration, _throughputDeadlineTime: Duration,
_executorServiceFactoryProvider: ExecutorServiceFactoryProvider, _executorServiceFactoryProvider: ExecutorServiceFactoryProvider,
_shutdownTimeout: FiniteDuration) _shutdownTimeout: FiniteDuration
)
extends Dispatcher(_configurator, _id, _throughput, _throughputDeadlineTime, _executorServiceFactoryProvider, _shutdownTimeout) { extends Dispatcher(_configurator, _id, _throughput, _throughputDeadlineTime, _executorServiceFactoryProvider, _shutdownTimeout) {
override protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope): Unit = { override protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope): Unit = {
@ -147,6 +149,7 @@ object TellOnlyBenchmark {
config.getInt("throughput"), config.getInt("throughput"),
config.getNanosDuration("throughput-deadline-time"), config.getNanosDuration("throughput-deadline-time"),
configureExecutor(), configureExecutor(),
config.getMillisDuration("shutdown-timeout")) config.getMillisDuration("shutdown-timeout")
)
} }
} }

View file

@ -28,7 +28,8 @@ class HttpBenchmark {
""" """
akka { akka {
loglevel = "ERROR" loglevel = "ERROR"
}""".stripMargin).withFallback(ConfigFactory.load()) }""".stripMargin
).withFallback(ConfigFactory.load())
implicit val system = ActorSystem("HttpBenchmark", config) implicit val system = ActorSystem("HttpBenchmark", config)
implicit val materializer = ActorMaterializer() implicit val materializer = ActorMaterializer()

View file

@ -101,7 +101,8 @@ class LevelDbBatchingBenchmark {
val storageLocations = List( val storageLocations = List(
"akka.persistence.journal.leveldb.dir", "akka.persistence.journal.leveldb.dir",
"akka.persistence.journal.leveldb-shared.store.dir", "akka.persistence.journal.leveldb-shared.store.dir",
"akka.persistence.snapshot-store.local.dir").map(s new File(sys.settings.config.getString(s))) "akka.persistence.snapshot-store.local.dir"
).map(s new File(sys.settings.config.getString(s)))
storageLocations.foreach(FileUtils.deleteDirectory) storageLocations.foreach(FileUtils.deleteDirectory)
} }

View file

@ -32,7 +32,8 @@ class PersistentActorDeferBenchmark {
lazy val storageLocations = List( lazy val storageLocations = List(
"akka.persistence.journal.leveldb.dir", "akka.persistence.journal.leveldb.dir",
"akka.persistence.journal.leveldb-shared.store.dir", "akka.persistence.journal.leveldb-shared.store.dir",
"akka.persistence.snapshot-store.local.dir").map(s new File(system.settings.config.getString(s))) "akka.persistence.snapshot-store.local.dir"
).map(s new File(system.settings.config.getString(s)))
var system: ActorSystem = _ var system: ActorSystem = _

View file

@ -21,7 +21,8 @@ class PersistentActorThroughputBenchmark {
lazy val storageLocations = List( lazy val storageLocations = List(
"akka.persistence.journal.leveldb.dir", "akka.persistence.journal.leveldb.dir",
"akka.persistence.journal.leveldb-shared.store.dir", "akka.persistence.journal.leveldb-shared.store.dir",
"akka.persistence.snapshot-store.local.dir").map(s new File(system.settings.config.getString(s))) "akka.persistence.snapshot-store.local.dir"
).map(s new File(system.settings.config.getString(s)))
var system: ActorSystem = _ var system: ActorSystem = _

View file

@ -22,7 +22,8 @@ class PersistentActorWithAtLeastOnceDeliveryBenchmark {
lazy val storageLocations = List( lazy val storageLocations = List(
"akka.persistence.journal.leveldb.dir", "akka.persistence.journal.leveldb.dir",
"akka.persistence.journal.leveldb-shared.store.dir", "akka.persistence.journal.leveldb-shared.store.dir",
"akka.persistence.snapshot-store.local.dir").map(s new File(system.settings.config.getString(s))) "akka.persistence.snapshot-store.local.dir"
).map(s new File(system.settings.config.getString(s)))
var system: ActorSystem = _ var system: ActorSystem = _

View file

@ -47,7 +47,8 @@ class FlowMapBenchmark {
type = akka.testkit.CallingThreadDispatcherConfigurator type = akka.testkit.CallingThreadDispatcherConfigurator
} }
} }
}""".stripMargin).withFallback(ConfigFactory.load()) }""".stripMargin
).withFallback(ConfigFactory.load())
implicit val system = ActorSystem("test", config) implicit val system = ActorSystem("test", config)

View file

@ -7,7 +7,6 @@ import akka.stream.impl.fusing.GraphInterpreter.{ DownstreamBoundaryStageLogic,
import akka.stream.stage._ import akka.stream.stage._
import org.openjdk.jmh.annotations._ import org.openjdk.jmh.annotations._
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
@State(Scope.Benchmark) @State(Scope.Benchmark)

View file

@ -43,14 +43,12 @@ object MaterializationBenchmark {
val graphWithNestedImportsBuilder = (numOfNestedGraphs: Int) => { val graphWithNestedImportsBuilder = (numOfNestedGraphs: Int) => {
var flow: Graph[FlowShape[Unit, Unit], NotUsed] = Flow[Unit].map(identity) var flow: Graph[FlowShape[Unit, Unit], NotUsed] = Flow[Unit].map(identity)
for (_ <- 1 to numOfNestedGraphs) { for (_ <- 1 to numOfNestedGraphs) {
flow = GraphDSL.create(flow) { b flow = GraphDSL.create(flow) { b flow
flow
FlowShape(flow.in, flow.out) FlowShape(flow.in, flow.out)
} }
} }
RunnableGraph.fromGraph(GraphDSL.create(flow) { implicit b RunnableGraph.fromGraph(GraphDSL.create(flow) { implicit b flow
flow
import GraphDSL.Implicits._ import GraphDSL.Implicits._
Source.single(()) ~> flow ~> Sink.ignore Source.single(()) ~> flow ~> Sink.ignore
ClosedShape ClosedShape
@ -58,8 +56,7 @@ object MaterializationBenchmark {
} }
val graphWithImportedFlowBuilder = (numOfFlows: Int) => val graphWithImportedFlowBuilder = (numOfFlows: Int) =>
RunnableGraph.fromGraph(GraphDSL.create(Source.single(())) { implicit b RunnableGraph.fromGraph(GraphDSL.create(Source.single(())) { implicit b source
source
import GraphDSL.Implicits._ import GraphDSL.Implicits._
val flow = Flow[Unit].map(identity) val flow = Flow[Unit].map(identity)
var out: Outlet[Unit] = source.out var out: Outlet[Unit] = source.out
@ -106,15 +103,12 @@ class MaterializationBenchmark {
@Benchmark @Benchmark
def flow_with_map(): Unit = flowWithMap.run() def flow_with_map(): Unit = flowWithMap.run()
@Benchmark @Benchmark
def graph_with_junctions(): Unit = graphWithJunctions.run() def graph_with_junctions(): Unit = graphWithJunctions.run()
@Benchmark @Benchmark
def graph_with_nested_imports(): Unit = graphWithNestedImports.run() def graph_with_nested_imports(): Unit = graphWithNestedImports.run()
@Benchmark @Benchmark
def graph_with_imported_flow(): Unit = graphWithImportedFlow.run() def graph_with_imported_flow(): Unit = graphWithImportedFlow.run()
} }

View file

@ -163,7 +163,7 @@ private[camel] class ProducerRegistrar(activationTracker: ActorRef) extends Acto
try { try {
val endpoint = camelContext.getEndpoint(endpointUri) val endpoint = camelContext.getEndpoint(endpointUri)
val processor = new SendProcessor(endpoint) val processor = new SendProcessor(endpoint)
camelObjects = camelObjects.updated(producer, endpoint -> processor) camelObjects = camelObjects.updated(producer, endpoint processor)
// if this throws, the supervisor stops the producer and de-registers it on termination // if this throws, the supervisor stops the producer and de-registers it on termination
processor.start() processor.start()
producer ! CamelProducerObjects(endpoint, processor) producer ! CamelProducerObjects(endpoint, processor)

View file

@ -49,7 +49,8 @@ private[camel] class ActorComponent(camel: Camel, system: ActorSystem) extends D
* <code>[actorPath]?[options]%s</code>, * <code>[actorPath]?[options]%s</code>,
* where <code>[actorPath]</code> refers to the actor path to the actor. * where <code>[actorPath]</code> refers to the actor path to the actor.
*/ */
private[camel] class ActorEndpoint(uri: String, private[camel] class ActorEndpoint(
uri: String,
comp: ActorComponent, comp: ActorComponent,
val path: ActorEndpointPath, val path: ActorEndpointPath,
val camel: Camel) extends DefaultEndpoint(uri, comp) with ActorEndpointConfig { val camel: Camel) extends DefaultEndpoint(uri, comp) with ActorEndpointConfig {
@ -174,7 +175,7 @@ private[camel] class ActorProducer(val endpoint: ActorEndpoint, camel: Camel) ex
path.findActorIn(camel.system) getOrElse (throw new ActorNotRegisteredException(path.actorPath)) path.findActorIn(camel.system) getOrElse (throw new ActorNotRegisteredException(path.actorPath))
private[this] def messageFor(exchange: CamelExchangeAdapter) = private[this] def messageFor(exchange: CamelExchangeAdapter) =
exchange.toRequestMessage(Map(CamelMessage.MessageExchangeId -> exchange.getExchangeId)) exchange.toRequestMessage(Map(CamelMessage.MessageExchangeId exchange.getExchangeId))
} }
/** /**

View file

@ -23,16 +23,21 @@ import java.util.concurrent.TimeUnit;
public class CustomRouteTest extends JUnitSuite { public class CustomRouteTest extends JUnitSuite {
@Rule @Rule
public AkkaJUnitActorSystemResource actorSystemResource = public AkkaJUnitActorSystemResource actorSystemResource = new AkkaJUnitActorSystemResource("CustomRouteTest");
new AkkaJUnitActorSystemResource("CustomRouteTest");
private final ActorSystem system = actorSystemResource.getSystem(); private ActorSystem system = null;
private Camel camel = (Camel) CamelExtension.get(system); private Camel camel = null;
public static class MyActor extends UntypedActor { public static class MyActor extends UntypedActor {
@Override public void onReceive(Object o) {} @Override public void onReceive(Object o) {}
} }
@Before
public void beforeEach() {
system = actorSystemResource.getSystem();
camel = (Camel) CamelExtension.get(system);
}
@Test @Test
public void testCustomProducerRoute() throws Exception { public void testCustomProducerRoute() throws Exception {
MockEndpoint mockEndpoint = camel.context().getEndpoint("mock:mockProducer", MockEndpoint.class); MockEndpoint mockEndpoint = camel.context().getEndpoint("mock:mockProducer", MockEndpoint.class);
@ -118,6 +123,7 @@ public class CustomRouteTest extends JUnitSuite {
private void assertMockEndpoint(MockEndpoint mockEndpoint) throws InterruptedException { private void assertMockEndpoint(MockEndpoint mockEndpoint) throws InterruptedException {
mockEndpoint.expectedMessageCount(1); mockEndpoint.expectedMessageCount(1);
mockEndpoint.expectedMessagesMatches(new Predicate() { mockEndpoint.expectedMessagesMatches(new Predicate() {
@Override
public boolean matches(Exchange exchange) { public boolean matches(Exchange exchange) {
return exchange.getIn().getBody().equals("test"); return exchange.getIn().getBody().equals("test");
} }
@ -126,8 +132,8 @@ public class CustomRouteTest extends JUnitSuite {
} }
public static class CustomRouteBuilder extends RouteBuilder { public static class CustomRouteBuilder extends RouteBuilder {
private String uri; private final String uri;
private String fromUri; private final String fromUri;
public CustomRouteBuilder(String from, String to) { public CustomRouteBuilder(String from, String to) {
fromUri = from; fromUri = from;
@ -164,7 +170,7 @@ public class CustomRouteTest extends JUnitSuite {
} }
public static class EndpointProducer extends UntypedProducerActor { public static class EndpointProducer extends UntypedProducerActor {
private String uri; private final String uri;
public EndpointProducer(String uri) { public EndpointProducer(String uri) {
this.uri = uri; this.uri = uri;
@ -192,8 +198,8 @@ public class CustomRouteTest extends JUnitSuite {
} }
public static class TestAckConsumer extends UntypedConsumerActor { public static class TestAckConsumer extends UntypedConsumerActor {
private String myuri; private final String myuri;
private String to; private final String to;
public TestAckConsumer(String uri, String to){ public TestAckConsumer(String uri, String to){
myuri = uri; myuri = uri;

View file

@ -50,17 +50,17 @@ class CamelExchangeAdapterTest extends FunSuite with SharedCamelSystem {
test("mustCreateRequestMessageFromInMessage") { test("mustCreateRequestMessageFromInMessage") {
val m = sampleInOnly.toRequestMessage val m = sampleInOnly.toRequestMessage
assert(m === CamelMessage("test-in", Map("key-in" -> "val-in"))) assert(m === CamelMessage("test-in", Map("key-in" "val-in")))
} }
test("mustCreateResponseMessageFromInMessage") { test("mustCreateResponseMessageFromInMessage") {
val m = sampleInOnly.toResponseMessage val m = sampleInOnly.toResponseMessage
assert(m === CamelMessage("test-in", Map("key-in" -> "val-in"))) assert(m === CamelMessage("test-in", Map("key-in" "val-in")))
} }
test("mustCreateResponseMessageFromOutMessage") { test("mustCreateResponseMessageFromOutMessage") {
val m = sampleInOut.toResponseMessage val m = sampleInOut.toResponseMessage
assert(m === CamelMessage("test-out", Map("key-out" -> "val-out"))) assert(m === CamelMessage("test-out", Map("key-out" "val-out")))
} }
test("mustCreateFailureMessageFromExceptionAndInMessage") { test("mustCreateFailureMessageFromExceptionAndInMessage") {
@ -82,30 +82,30 @@ class CamelExchangeAdapterTest extends FunSuite with SharedCamelSystem {
} }
test("mustCreateRequestMessageFromInMessageWithAdditionalHeader") { test("mustCreateRequestMessageFromInMessageWithAdditionalHeader") {
val m = sampleInOnly.toRequestMessage(Map("x" -> "y")) val m = sampleInOnly.toRequestMessage(Map("x" "y"))
assert(m === CamelMessage("test-in", Map("key-in" -> "val-in", "x" -> "y"))) assert(m === CamelMessage("test-in", Map("key-in" "val-in", "x" "y")))
} }
test("mustCreateResponseMessageFromInMessageWithAdditionalHeader") { test("mustCreateResponseMessageFromInMessageWithAdditionalHeader") {
val m = sampleInOnly.toResponseMessage(Map("x" -> "y")) val m = sampleInOnly.toResponseMessage(Map("x" "y"))
assert(m === CamelMessage("test-in", Map("key-in" -> "val-in", "x" -> "y"))) assert(m === CamelMessage("test-in", Map("key-in" "val-in", "x" "y")))
} }
test("mustCreateResponseMessageFromOutMessageWithAdditionalHeader") { test("mustCreateResponseMessageFromOutMessageWithAdditionalHeader") {
val m = sampleInOut.toResponseMessage(Map("x" -> "y")) val m = sampleInOut.toResponseMessage(Map("x" "y"))
assert(m === CamelMessage("test-out", Map("key-out" -> "val-out", "x" -> "y"))) assert(m === CamelMessage("test-out", Map("key-out" "val-out", "x" "y")))
} }
test("mustCreateFailureMessageFromExceptionAndInMessageWithAdditionalHeader") { test("mustCreateFailureMessageFromExceptionAndInMessageWithAdditionalHeader") {
val e1 = sampleInOnly val e1 = sampleInOnly
e1.setException(new Exception("test1")) e1.setException(new Exception("test1"))
assert(e1.toAkkaCamelException.getMessage === "test1") assert(e1.toAkkaCamelException.getMessage === "test1")
val headers = e1.toAkkaCamelException(Map("x" -> "y")).headers val headers = e1.toAkkaCamelException(Map("x" "y")).headers
assert(headers("key-in") === "val-in") assert(headers("key-in") === "val-in")
assert(headers("x") === "y") assert(headers("x") === "y")
assert(e1.toFailureMessage.cause.getMessage === "test1") assert(e1.toFailureMessage.cause.getMessage === "test1")
val failureHeaders = e1.toFailureResult(Map("x" -> "y")).headers val failureHeaders = e1.toFailureResult(Map("x" "y")).headers
assert(failureHeaders("key-in") === "val-in") assert(failureHeaders("key-in") === "val-in")
assert(failureHeaders("x") === "y") assert(failureHeaders("x") === "y")
@ -115,11 +115,11 @@ class CamelExchangeAdapterTest extends FunSuite with SharedCamelSystem {
val e1 = sampleInOut val e1 = sampleInOut
e1.setException(new Exception("test2")) e1.setException(new Exception("test2"))
assert(e1.toAkkaCamelException.getMessage === "test2") assert(e1.toAkkaCamelException.getMessage === "test2")
val headers = e1.toAkkaCamelException(Map("x" -> "y")).headers val headers = e1.toAkkaCamelException(Map("x" "y")).headers
assert(headers("key-out") === "val-out") assert(headers("key-out") === "val-out")
assert(headers("x") === "y") assert(headers("x") === "y")
assert(e1.toFailureMessage.cause.getMessage === "test2") assert(e1.toFailureMessage.cause.getMessage === "test2")
val failureHeaders = e1.toFailureResult(Map("x" -> "y")).headers val failureHeaders = e1.toFailureResult(Map("x" "y")).headers
assert(failureHeaders("key-out") === "val-out") assert(failureHeaders("key-out") === "val-out")
assert(failureHeaders("x") === "y") assert(failureHeaders("x") === "y")
} }

View file

@ -25,7 +25,7 @@ class CamelMessageTest extends Matchers with WordSpecLike with SharedCamelSystem
message.setExchange(new DefaultExchange(camel.context)) message.setExchange(new DefaultExchange(camel.context))
val attachmentToAdd = new DataHandler(new URL("https://another.url")) val attachmentToAdd = new DataHandler(new URL("https://another.url"))
CamelMessage.copyContent(new CamelMessage("body", Map("key" -> "baz"), Map("key" -> attachmentToAdd)), message) CamelMessage.copyContent(new CamelMessage("body", Map("key" "baz"), Map("key" attachmentToAdd)), message)
assert(message.getBody === "body") assert(message.getBody === "body")
assert(message.getHeader("foo") === "bar") assert(message.getHeader("foo") === "bar")

View file

@ -67,8 +67,8 @@ class ConcurrentActivationTest extends WordSpec with Matchers with NonSharedCame
} }
val (activatedConsumerNames, activatedProducerNames) = partitionNames(activations) val (activatedConsumerNames, activatedProducerNames) = partitionNames(activations)
val (deactivatedConsumerNames, deactivatedProducerNames) = partitionNames(deactivations) val (deactivatedConsumerNames, deactivatedProducerNames) = partitionNames(deactivations)
assertContainsSameElements(activatedConsumerNames -> deactivatedConsumerNames) assertContainsSameElements(activatedConsumerNames deactivatedConsumerNames)
assertContainsSameElements(activatedProducerNames -> deactivatedProducerNames) assertContainsSameElements(activatedProducerNames deactivatedProducerNames)
} finally { } finally {
system.eventStream.publish(TestEvent.UnMute(eventFilter)) system.eventStream.publish(TestEvent.UnMute(eventFilter))
} }
@ -95,7 +95,7 @@ class ConsumerBroadcast(promise: Promise[(Future[List[List[ActorRef]]], Future[L
val routee = context.actorOf(Props(classOf[Registrar], i, number, activationListPromise, deactivationListPromise), "registrar-" + i) val routee = context.actorOf(Props(classOf[Registrar], i, number, activationListPromise, deactivationListPromise), "registrar-" + i)
routee.path.toString routee.path.toString
} }
promise.success(Future.sequence(allActivationFutures) -> Future.sequence(allDeactivationFutures)) promise.success(Future.sequence(allActivationFutures) Future.sequence(allDeactivationFutures))
broadcaster = Some(context.actorOf(BroadcastGroup(routeePaths).props(), "registrarRouter")) broadcaster = Some(context.actorOf(BroadcastGroup(routeePaths).props(), "registrarRouter"))
case reg: Any case reg: Any

View file

@ -24,31 +24,31 @@ class MessageScalaTest extends FunSuite with Matchers with SharedCamelSystem {
} }
test("mustConvertDoubleHeaderToString") { test("mustConvertDoubleHeaderToString") {
val message = CamelMessage("test", Map("test" -> 1.4)) val message = CamelMessage("test", Map("test" 1.4))
message.headerAs[String]("test").get should ===("1.4") message.headerAs[String]("test").get should ===("1.4")
} }
test("mustReturnSubsetOfHeaders") { test("mustReturnSubsetOfHeaders") {
val message = CamelMessage("test", Map("A" -> "1", "B" -> "2")) val message = CamelMessage("test", Map("A" "1", "B" "2"))
message.headers(Set("B")) should ===(Map("B" -> "2")) message.headers(Set("B")) should ===(Map("B" "2"))
} }
test("mustTransformBodyAndPreserveHeaders") { test("mustTransformBodyAndPreserveHeaders") {
CamelMessage("a", Map("A" -> "1")).mapBody((body: String) body + "b") should ===(CamelMessage("ab", Map("A" -> "1"))) CamelMessage("a", Map("A" "1")).mapBody((body: String) body + "b") should ===(CamelMessage("ab", Map("A" "1")))
} }
test("mustConvertBodyAndPreserveHeaders") { test("mustConvertBodyAndPreserveHeaders") {
CamelMessage(1.4, Map("A" -> "1")).withBodyAs[String] should ===(CamelMessage("1.4", Map("A" -> "1"))) CamelMessage(1.4, Map("A" "1")).withBodyAs[String] should ===(CamelMessage("1.4", Map("A" "1")))
} }
test("mustSetBodyAndPreserveHeaders") { test("mustSetBodyAndPreserveHeaders") {
CamelMessage("test1", Map("A" -> "1")).copy(body = "test2") should ===( CamelMessage("test1", Map("A" "1")).copy(body = "test2") should ===(
CamelMessage("test2", Map("A" -> "1"))) CamelMessage("test2", Map("A" "1")))
} }
test("mustSetHeadersAndPreserveBody") { test("mustSetHeadersAndPreserveBody") {
CamelMessage("test1", Map("A" -> "1")).copy(headers = Map("C" -> "3")) should ===( CamelMessage("test1", Map("A" "1")).copy(headers = Map("C" "3")) should ===(
CamelMessage("test1", Map("C" -> "3"))) CamelMessage("test1", Map("C" "3")))
} }
test("mustBeAbleToReReadStreamCacheBody") { test("mustBeAbleToReReadStreamCacheBody") {

View file

@ -45,9 +45,9 @@ class ProducerFeatureTest extends TestKit(ActorSystem("ProducerFeatureTest", Akk
"01 produce a message and receive normal response" in { "01 produce a message and receive normal response" in {
val producer = system.actorOf(Props(new TestProducer("direct:producer-test-2", true)), name = "01-direct-producer-2") val producer = system.actorOf(Props(new TestProducer("direct:producer-test-2", true)), name = "01-direct-producer-2")
val message = CamelMessage("test", Map(CamelMessage.MessageExchangeId -> "123")) val message = CamelMessage("test", Map(CamelMessage.MessageExchangeId "123"))
producer.tell(message, testActor) producer.tell(message, testActor)
expectMsg(CamelMessage("received TEST", Map(CamelMessage.MessageExchangeId -> "123"))) expectMsg(CamelMessage("received TEST", Map(CamelMessage.MessageExchangeId "123")))
} }
"02 produce a message and receive failure response" in { "02 produce a message and receive failure response" in {
@ -72,13 +72,13 @@ class ProducerFeatureTest extends TestKit(ActorSystem("ProducerFeatureTest", Akk
supervisor.tell(Props(new TestProducer("direct:producer-test-2")), testActor) supervisor.tell(Props(new TestProducer("direct:producer-test-2")), testActor)
val producer = receiveOne(timeoutDuration).asInstanceOf[ActorRef] val producer = receiveOne(timeoutDuration).asInstanceOf[ActorRef]
val message = CamelMessage("fail", Map(CamelMessage.MessageExchangeId -> "123")) val message = CamelMessage("fail", Map(CamelMessage.MessageExchangeId "123"))
filterEvents(EventFilter[AkkaCamelException](occurrences = 1)) { filterEvents(EventFilter[AkkaCamelException](occurrences = 1)) {
producer.tell(message, testActor) producer.tell(message, testActor)
expectMsgPF(timeoutDuration) { expectMsgPF(timeoutDuration) {
case Failure(e: AkkaCamelException) case Failure(e: AkkaCamelException)
e.getMessage should ===("failure") e.getMessage should ===("failure")
e.headers should ===(Map(CamelMessage.MessageExchangeId -> "123")) e.headers should ===(Map(CamelMessage.MessageExchangeId "123"))
} }
} }
Await.ready(latch, timeoutDuration) Await.ready(latch, timeoutDuration)
@ -106,21 +106,21 @@ class ProducerFeatureTest extends TestKit(ActorSystem("ProducerFeatureTest", Akk
"10 produce message to direct:producer-test-3 and receive normal response" in { "10 produce message to direct:producer-test-3 and receive normal response" in {
val producer = system.actorOf(Props(new TestProducer("direct:producer-test-3")), name = "10-direct-producer-test-3") val producer = system.actorOf(Props(new TestProducer("direct:producer-test-3")), name = "10-direct-producer-test-3")
val message = CamelMessage("test", Map(CamelMessage.MessageExchangeId -> "123")) val message = CamelMessage("test", Map(CamelMessage.MessageExchangeId "123"))
producer.tell(message, testActor) producer.tell(message, testActor)
expectMsg(CamelMessage("received test", Map(CamelMessage.MessageExchangeId -> "123"))) expectMsg(CamelMessage("received test", Map(CamelMessage.MessageExchangeId "123")))
} }
"11 produce message to direct:producer-test-3 and receive failure response" in { "11 produce message to direct:producer-test-3 and receive failure response" in {
val producer = system.actorOf(Props(new TestProducer("direct:producer-test-3")), name = "11-direct-producer-test-3-receive-failure") val producer = system.actorOf(Props(new TestProducer("direct:producer-test-3")), name = "11-direct-producer-test-3-receive-failure")
val message = CamelMessage("fail", Map(CamelMessage.MessageExchangeId -> "123")) val message = CamelMessage("fail", Map(CamelMessage.MessageExchangeId "123"))
filterEvents(EventFilter[AkkaCamelException](occurrences = 1)) { filterEvents(EventFilter[AkkaCamelException](occurrences = 1)) {
producer.tell(message, testActor) producer.tell(message, testActor)
expectMsgPF(timeoutDuration) { expectMsgPF(timeoutDuration) {
case Failure(e: AkkaCamelException) case Failure(e: AkkaCamelException)
e.getMessage should ===("failure") e.getMessage should ===("failure")
e.headers should ===(Map(CamelMessage.MessageExchangeId -> "123")) e.headers should ===(Map(CamelMessage.MessageExchangeId "123"))
} }
} }
} }
@ -128,22 +128,22 @@ class ProducerFeatureTest extends TestKit(ActorSystem("ProducerFeatureTest", Akk
"12 produce message, forward normal response of direct:producer-test-2 to a replying target actor and receive response" in { "12 produce message, forward normal response of direct:producer-test-2 to a replying target actor and receive response" in {
val target = system.actorOf(Props[ReplyingForwardTarget], name = "12-reply-forwarding-target") val target = system.actorOf(Props[ReplyingForwardTarget], name = "12-reply-forwarding-target")
val producer = system.actorOf(Props(new TestForwarder("direct:producer-test-2", target)), name = "12-direct-producer-test-2-forwarder") val producer = system.actorOf(Props(new TestForwarder("direct:producer-test-2", target)), name = "12-direct-producer-test-2-forwarder")
val message = CamelMessage("test", Map(CamelMessage.MessageExchangeId -> "123")) val message = CamelMessage("test", Map(CamelMessage.MessageExchangeId "123"))
producer.tell(message, testActor) producer.tell(message, testActor)
expectMsg(CamelMessage("received test", Map(CamelMessage.MessageExchangeId -> "123", "test" -> "result"))) expectMsg(CamelMessage("received test", Map(CamelMessage.MessageExchangeId "123", "test" "result")))
} }
"13 produce message, forward failure response of direct:producer-test-2 to a replying target actor and receive response" in { "13 produce message, forward failure response of direct:producer-test-2 to a replying target actor and receive response" in {
val target = system.actorOf(Props[ReplyingForwardTarget], name = "13-reply-forwarding-target") val target = system.actorOf(Props[ReplyingForwardTarget], name = "13-reply-forwarding-target")
val producer = system.actorOf(Props(new TestForwarder("direct:producer-test-2", target)), name = "13-direct-producer-test-2-forwarder-failure") val producer = system.actorOf(Props(new TestForwarder("direct:producer-test-2", target)), name = "13-direct-producer-test-2-forwarder-failure")
val message = CamelMessage("fail", Map(CamelMessage.MessageExchangeId -> "123")) val message = CamelMessage("fail", Map(CamelMessage.MessageExchangeId "123"))
filterEvents(EventFilter[AkkaCamelException](occurrences = 1)) { filterEvents(EventFilter[AkkaCamelException](occurrences = 1)) {
producer.tell(message, testActor) producer.tell(message, testActor)
expectMsgPF(timeoutDuration) { expectMsgPF(timeoutDuration) {
case Failure(e: AkkaCamelException) case Failure(e: AkkaCamelException)
e.getMessage should ===("failure") e.getMessage should ===("failure")
e.headers should ===(Map(CamelMessage.MessageExchangeId -> "123", "test" -> "failure")) e.headers should ===(Map(CamelMessage.MessageExchangeId "123", "test" "failure"))
} }
} }
} }
@ -170,23 +170,23 @@ class ProducerFeatureTest extends TestKit(ActorSystem("ProducerFeatureTest", Akk
"16 produce message, forward normal response from direct:producer-test-3 to a replying target actor and receive response" in { "16 produce message, forward normal response from direct:producer-test-3 to a replying target actor and receive response" in {
val target = system.actorOf(Props[ReplyingForwardTarget], name = "16-reply-forwarding-target") val target = system.actorOf(Props[ReplyingForwardTarget], name = "16-reply-forwarding-target")
val producer = system.actorOf(Props(new TestForwarder("direct:producer-test-3", target)), name = "16-direct-producer-test-3-to-replying-actor") val producer = system.actorOf(Props(new TestForwarder("direct:producer-test-3", target)), name = "16-direct-producer-test-3-to-replying-actor")
val message = CamelMessage("test", Map(CamelMessage.MessageExchangeId -> "123")) val message = CamelMessage("test", Map(CamelMessage.MessageExchangeId "123"))
producer.tell(message, testActor) producer.tell(message, testActor)
expectMsg(CamelMessage("received test", Map(CamelMessage.MessageExchangeId -> "123", "test" -> "result"))) expectMsg(CamelMessage("received test", Map(CamelMessage.MessageExchangeId "123", "test" "result")))
} }
"17 produce message, forward failure response from direct:producer-test-3 to a replying target actor and receive response" in { "17 produce message, forward failure response from direct:producer-test-3 to a replying target actor and receive response" in {
val target = system.actorOf(Props[ReplyingForwardTarget], name = "17-reply-forwarding-target") val target = system.actorOf(Props[ReplyingForwardTarget], name = "17-reply-forwarding-target")
val producer = system.actorOf(Props(new TestForwarder("direct:producer-test-3", target)), name = "17-direct-producer-test-3-forward-failure") val producer = system.actorOf(Props(new TestForwarder("direct:producer-test-3", target)), name = "17-direct-producer-test-3-forward-failure")
val message = CamelMessage("fail", Map(CamelMessage.MessageExchangeId -> "123")) val message = CamelMessage("fail", Map(CamelMessage.MessageExchangeId "123"))
filterEvents(EventFilter[AkkaCamelException](occurrences = 1)) { filterEvents(EventFilter[AkkaCamelException](occurrences = 1)) {
producer.tell(message, testActor) producer.tell(message, testActor)
expectMsgPF(timeoutDuration) { expectMsgPF(timeoutDuration) {
case Failure(e: AkkaCamelException) case Failure(e: AkkaCamelException)
e.getMessage should ===("failure") e.getMessage should ===("failure")
e.headers should ===(Map(CamelMessage.MessageExchangeId -> "123", "test" -> "failure")) e.headers should ===(Map(CamelMessage.MessageExchangeId "123", "test" "failure"))
} }
} }
} }
@ -324,10 +324,10 @@ object ProducerFeatureTest {
class ReplyingForwardTarget extends Actor { class ReplyingForwardTarget extends Actor {
def receive = { def receive = {
case msg: CamelMessage case msg: CamelMessage
context.sender() ! (msg.copy(headers = msg.headers + ("test" -> "result"))) context.sender() ! (msg.copy(headers = msg.headers + ("test" "result")))
case msg: akka.actor.Status.Failure case msg: akka.actor.Status.Failure
msg.cause match { msg.cause match {
case e: AkkaCamelException context.sender() ! Status.Failure(new AkkaCamelException(e, e.headers + ("test" -> "failure"))) case e: AkkaCamelException context.sender() ! Status.Failure(new AkkaCamelException(e, e.headers + ("test" "failure")))
} }
} }
} }

View file

@ -48,7 +48,7 @@ private[camel] object TestSupport {
} }
trait SharedCamelSystem extends BeforeAndAfterAll { this: Suite trait SharedCamelSystem extends BeforeAndAfterAll { this: Suite
implicit lazy val system = ActorSystem("test", AkkaSpec.testConf) implicit lazy val system = ActorSystem("SharedCamelSystem", AkkaSpec.testConf)
implicit lazy val camel = CamelExtension(system) implicit lazy val camel = CamelExtension(system)
abstract override protected def afterAll() { abstract override protected def afterAll() {
@ -63,7 +63,7 @@ private[camel] object TestSupport {
override protected def beforeEach() { override protected def beforeEach() {
super.beforeEach() super.beforeEach()
system = ActorSystem("test", AkkaSpec.testConf) system = ActorSystem("NonSharedCamelSystem", AkkaSpec.testConf)
camel = CamelExtension(system) camel = CamelExtension(system)
} }

View file

@ -34,10 +34,10 @@ class UntypedProducerTest extends WordSpec with Matchers with BeforeAndAfterAll
"produce a message and receive a normal response" in { "produce a message and receive a normal response" in {
val producer = system.actorOf(Props[SampleUntypedReplyingProducer], name = "sample-untyped-replying-producer") val producer = system.actorOf(Props[SampleUntypedReplyingProducer], name = "sample-untyped-replying-producer")
val message = CamelMessage("test", Map(CamelMessage.MessageExchangeId -> "123")) val message = CamelMessage("test", Map(CamelMessage.MessageExchangeId "123"))
val future = producer.ask(message)(timeout) val future = producer.ask(message)(timeout)
val expected = CamelMessage("received test", Map(CamelMessage.MessageExchangeId -> "123")) val expected = CamelMessage("received test", Map(CamelMessage.MessageExchangeId "123"))
Await.result(future, timeout) match { Await.result(future, timeout) match {
case result: CamelMessage result should ===(expected) case result: CamelMessage result should ===(expected)
case unexpected fail("Actor responded with unexpected message:" + unexpected) case unexpected fail("Actor responded with unexpected message:" + unexpected)
@ -48,14 +48,14 @@ class UntypedProducerTest extends WordSpec with Matchers with BeforeAndAfterAll
"produce a message and receive a failure response" in { "produce a message and receive a failure response" in {
val producer = system.actorOf(Props[SampleUntypedReplyingProducer], name = "sample-untyped-replying-producer-failure") val producer = system.actorOf(Props[SampleUntypedReplyingProducer], name = "sample-untyped-replying-producer-failure")
val message = CamelMessage("fail", Map(CamelMessage.MessageExchangeId -> "123")) val message = CamelMessage("fail", Map(CamelMessage.MessageExchangeId "123"))
filterEvents(EventFilter[AkkaCamelException](occurrences = 1)) { filterEvents(EventFilter[AkkaCamelException](occurrences = 1)) {
val future = producer.ask(message)(timeout).failed val future = producer.ask(message)(timeout).failed
Await.result(future, timeout) match { Await.result(future, timeout) match {
case e: AkkaCamelException case e: AkkaCamelException
e.getMessage should ===("failure") e.getMessage should ===("failure")
e.headers should ===(Map(CamelMessage.MessageExchangeId -> "123")) e.headers should ===(Map(CamelMessage.MessageExchangeId "123"))
case unexpected fail("Actor responded with unexpected message:" + unexpected) case unexpected fail("Actor responded with unexpected message:" + unexpected)
} }
} }

View file

@ -7,7 +7,7 @@ import akka.actor.{ Props, ActorSystem }
import akka.testkit.{ TimingTest, TestProbe, TestKit } import akka.testkit.{ TimingTest, TestProbe, TestKit }
import akka.camel.internal.ActivationProtocol._ import akka.camel.internal.ActivationProtocol._
class ActivationTrackerTest extends TestKit(ActorSystem("test")) with WordSpecLike with Matchers with BeforeAndAfterAll with BeforeAndAfterEach with GivenWhenThen { class ActivationTrackerTest extends TestKit(ActorSystem("ActivationTrackerTest")) with WordSpecLike with Matchers with BeforeAndAfterAll with BeforeAndAfterEach with GivenWhenThen {
override protected def afterAll() { shutdown() } override protected def afterAll() { shutdown() }

View file

@ -30,7 +30,7 @@ import akka.util.Timeout
import akka.actor._ import akka.actor._
import akka.testkit._ import akka.testkit._
class ActorProducerTest extends TestKit(ActorSystem("test")) with WordSpecLike with Matchers with ActorProducerFixture { class ActorProducerTest extends TestKit(ActorSystem("ActorProducerTest")) with WordSpecLike with Matchers with ActorProducerFixture {
implicit val timeout = Timeout(10 seconds) implicit val timeout = Timeout(10 seconds)
"ActorProducer" when { "ActorProducer" when {

View file

@ -153,13 +153,15 @@ private[metrics] class ClusterMetricsCollector extends Actor with ActorLogging {
/** /**
* Start periodic gossip to random nodes in cluster * Start periodic gossip to random nodes in cluster
*/ */
val gossipTask = scheduler.schedule(PeriodicTasksInitialDelay max CollectorGossipInterval, val gossipTask = scheduler.schedule(
PeriodicTasksInitialDelay max CollectorGossipInterval,
CollectorGossipInterval, self, GossipTick) CollectorGossipInterval, self, GossipTick)
/** /**
* Start periodic metrics collection * Start periodic metrics collection
*/ */
val sampleTask = scheduler.schedule(PeriodicTasksInitialDelay max CollectorSampleInterval, val sampleTask = scheduler.schedule(
PeriodicTasksInitialDelay max CollectorSampleInterval,
CollectorSampleInterval, self, MetricsTick) CollectorSampleInterval, self, MetricsTick)
override def preStart(): Unit = { override def preStart(): Unit = {

View file

@ -44,7 +44,7 @@ class ClusterMetricsExtension(system: ExtendedActorSystem) extends Extension {
* Supervision strategy. * Supervision strategy.
*/ */
private[metrics] val strategy = system.dynamicAccess.createInstanceFor[SupervisorStrategy]( private[metrics] val strategy = system.dynamicAccess.createInstanceFor[SupervisorStrategy](
SupervisorStrategyProvider, immutable.Seq(classOf[Config] -> SupervisorStrategyConfiguration)) SupervisorStrategyProvider, immutable.Seq(classOf[Config] SupervisorStrategyConfiguration))
.getOrElse { .getOrElse {
val log: LoggingAdapter = Logging(system, getClass.getName) val log: LoggingAdapter = Logging(system, getClass.getName)
log.error(s"Configured strategy provider ${SupervisorStrategyProvider} failed to load, using default ${classOf[ClusterMetricsStrategy].getName}.") log.error(s"Configured strategy provider ${SupervisorStrategyProvider} failed to load, using default ${classOf[ClusterMetricsStrategy].getName}.")

Some files were not shown because too many files have changed in this diff Show more