[
https://issues.apache.org/jira/browse/FLINK-13484?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16906152#comment-16906152
]
Gary Yao edited comment on FLINK-13484 at 8/13/19 12:57 PM:
------------------------------------------------------------
In my experiments, the test frequently fails due to {{AskTimeoutException}}.
The test attempts to run the job with a parallelism of 25 on 25 individual
TaskManager processes, each configured with 44 mb of heap space.
It seems to me that the VM on Travis CI cannot handle this number of
simultaneous TMs. Here is the relevant part of the JM log:
{noformat}
2019-08-12 13:11:34,185 INFO
org.apache.flink.runtime.executiongraph.ExecutionGraph - CHAIN Join
(Join at main(HighParallelismIterationsTestProgram.java:60)) -> Combine
(MIN(1), at main(HighParallelismIterationsTestProgram.java:62) (10/25)
(19811ac3816e2e2283596ecdff034321) switched from CREATED to SCHEDULED.
2019-08-12 13:11:37,694 INFO
org.apache.flink.runtime.executiongraph.ExecutionGraph - CHAIN Join
(Join at main(HighParallelismIterationsTestProgram.java:60)) -> Combine
(MIN(1), at main(HighParallelismIterationsTestProgram.java:62) (10/25)
(19811ac3816e2e2283596ecdff034321) switched from SCHEDULED to DEPLOYING.
2019-08-12 13:11:37,710 INFO
org.apache.flink.runtime.executiongraph.ExecutionGraph - Deploying CHAIN
Join (Join at main(HighParallelismIterationsTestProgram.java:60)) -> Combine
(MIN(1), at main(HighParallelismIterationsTestProgram.java:62) (10/25) (attempt
#0) to dadc9c817c5218f48538c3eaf1172788 @ localhost (dataPort=39211)
2019-08-12 13:11:47,735 INFO
org.apache.flink.runtime.executiongraph.ExecutionGraph - CHAIN Join
(Join at main(HighParallelismIterationsTestProgram.java:60)) -> Combine
(MIN(1), at main(HighParallelismIterationsTestProgram.java:62) (10/25)
(19811ac3816e2e2283596ecdff034321) switched from DEPLOYING to FAILED.
java.util.concurrent.CompletionException: akka.pattern.AskTimeoutException: Ask
timed out on
[Actor[akka.tcp://[email protected]:46641/user/taskmanager_0#925859671]] after
[10000 ms]. Message of type
[org.apache.flink.runtime.rpc.messages.RemoteRpcInvocation]. A typical reason
for `AskTimeoutException` is that the recipient actor didn't send a reply.
at
java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
at
java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
at
java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
at
java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
at
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
at
java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
at
org.apache.flink.runtime.concurrent.FutureUtils$1.onComplete(FutureUtils.java:871)
at akka.dispatch.OnComplete.internal(Future.scala:263)
at akka.dispatch.OnComplete.internal(Future.scala:261)
at akka.dispatch.japi$CallbackBridge.apply(Future.scala:191)
at akka.dispatch.japi$CallbackBridge.apply(Future.scala:188)
at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:36)
at
org.apache.flink.runtime.concurrent.Executors$DirectExecutionContext.execute(Executors.java:74)
at
scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
at
scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
at
akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:644)
at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:205)
at
scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
at
scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
at
scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
at
akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:328)
at
akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:279)
at
akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:283)
at
akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:235)
at java.lang.Thread.run(Thread.java:748)
{noformat}
As can be seen, the job fails due to a task deployment failure. The task is
deployed at 13:11:37 and fails at 13:11:47, which matches the default akka ask
timeout of 10s.
However, on TM side the task submission is received at 13:11:47:
{noformat}
2019-08-12 13:11:47,989 INFO
org.apache.flink.runtime.taskexecutor.TaskExecutor - Received task
CHAIN Join (Join at main(HighParallelismIterationsTestProgram.java:60)) ->
Combine (MIN(1), at main(HighParallelismIterationsTestProgram.java:62) (10/25).
2019-08-12 13:11:48,404 INFO org.apache.flink.runtime.taskmanager.Task
- CHAIN Join (Join at
main(HighParallelismIterationsTestProgram.java:60)) -> Combine (MIN(1), at
main(Hig
hParallelismIterationsTestProgram.java:62) (10/25)
(19811ac3816e2e2283596ecdff034321) switched from CREATED to DEPLOYING.
2019-08-12 13:11:48,422 INFO org.apache.flink.runtime.taskmanager.Task
- Creating FileSystem stream leak safety net for task CHAIN Join
(Join at main(HighParallelismIterationsT
estProgram.java:60)) -> Combine (MIN(1), at
main(HighParallelismIterationsTestProgram.java:62) (10/25)
(19811ac3816e2e2283596ecdff034321) [DEPLOYING]
2019-08-12 13:11:48,422 INFO org.apache.flink.runtime.taskmanager.Task
- Loading JAR files for task CHAIN Join (Join at
main(HighParallelismIterationsTestProgram.java:60)) -> Combine (MIN(1), at
main(HighParallelismIterationsTestProgram.java:62) (10/25)
(19811ac3816e2e2283596ecdff034321) [DEPLOYING].
2019-08-12 13:11:50,128 INFO org.apache.flink.runtime.taskmanager.Task
- Attempting to cancel task Partition (25/25)
(32294ea5064d9bb8b67fa7f848909a52).
2019-08-12 13:11:50,128 INFO org.apache.flink.runtime.taskmanager.Task
- Partition (25/25) (32294ea5064d9bb8b67fa7f848909a52) switched
from DEPLOYING to CANCELING.
{noformat}
No other exceptions can be found in the TM logs.
In addition, I have enabled verbose gc logging for TM and JM processes –
nothing suspicious can be found there.
Another problem is that if the Flink test job is not completed successfully,
the test script exits with code 0. We then assert that no exceptions are logged
in the JM and TM logs. It would be cleaner to exit the test script with the
same code that the Flink CLI returns, and maybe skip checking for exceptions in
the logs.
All in all, I would propose the following changes:
- Test should fail if Flink CLI exits with a code other than 0
- Start 2 TMs with 13 slots each instead of 25 TMs with 1 slot
was (Author: gjy):
In my experiments, the test frequently fails due to {{AskTimeoutException}}.
The test attempts to run the job with a parallelism of 25 on 25 individual
TaskManager processes, each configured with 44 mb of heap space.
It seems to me that the VM on Travis CI cannot handle this number of
simultaneous TMs. Here is the relevant part of the JM log:
{noformat}
2019-08-12 13:11:34,185 INFO
org.apache.flink.runtime.executiongraph.ExecutionGraph - CHAIN Join
(Join at main(HighParallelismIterationsTestProgram.java:60)) -> Combine
(MIN(1), at main(HighParallelismIterationsTestProgram.java:62) (10/25)
(19811ac3816e2e2283596ecdff034321) switched from CREATED to SCHEDULED.
2019-08-12 13:11:37,694 INFO
org.apache.flink.runtime.executiongraph.ExecutionGraph - CHAIN Join
(Join at main(HighParallelismIterationsTestProgram.java:60)) -> Combine
(MIN(1), at main(HighParallelismIterationsTestProgram.java:62) (10/25)
(19811ac3816e2e2283596ecdff034321) switched from SCHEDULED to DEPLOYING.
2019-08-12 13:11:37,710 INFO
org.apache.flink.runtime.executiongraph.ExecutionGraph - Deploying CHAIN
Join (Join at main(HighParallelismIterationsTestProgram.java:60)) -> Combine
(MIN(1), at main(HighParallelismIterationsTestProgram.java:62) (10/25) (attempt
#0) to dadc9c817c5218f48538c3eaf1172788 @ localhost (dataPort=39211)
2019-08-12 13:11:47,735 INFO
org.apache.flink.runtime.executiongraph.ExecutionGraph - CHAIN Join
(Join at main(HighParallelismIterationsTestProgram.java:60)) -> Combine
(MIN(1), at main(HighParallelismIterationsTestProgram.java:62) (10/25)
(19811ac3816e2e2283596ecdff034321) switched from DEPLOYING to FAILED.
java.util.concurrent.CompletionException: akka.pattern.AskTimeoutException: Ask
timed out on
[Actor[akka.tcp://[email protected]:46641/user/taskmanager_0#925859671]] after
[10000 ms]. Message of type
[org.apache.flink.runtime.rpc.messages.RemoteRpcInvocation]. A typical reason
for `AskTimeoutException` is that the recipient actor didn't send a reply.
at
java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
at
java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
at
java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
at
java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
at
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
at
java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
at
org.apache.flink.runtime.concurrent.FutureUtils$1.onComplete(FutureUtils.java:871)
at akka.dispatch.OnComplete.internal(Future.scala:263)
at akka.dispatch.OnComplete.internal(Future.scala:261)
at akka.dispatch.japi$CallbackBridge.apply(Future.scala:191)
at akka.dispatch.japi$CallbackBridge.apply(Future.scala:188)
at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:36)
at
org.apache.flink.runtime.concurrent.Executors$DirectExecutionContext.execute(Executors.java:74)
at
scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
at
scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
at
akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:644)
at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:205)
at
scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
at
scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
at
scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
at
akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:328)
at
akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:279)
at
akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:283)
at
akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:235)
at java.lang.Thread.run(Thread.java:748)
{noformat}
As can be seen, the job fails due to a task deployment failure. The task is
deployed at 13:11:37 and fails at 13:11:47, which matches the default akka ask
timeout of 10s.
However, on TM side the task submission is received at 13:11:47:
{noformat}
2019-08-12 13:11:47,989 INFO
org.apache.flink.runtime.taskexecutor.TaskExecutor - Received task
CHAIN Join (Join at main(HighParallelismIterationsTestProgram.java:60)) ->
Combine (MIN(1), at main(HighParallelismIterationsTestProgram.java:62) (10/25).
2019-08-12 13:11:48,404 INFO org.apache.flink.runtime.taskmanager.Task
- CHAIN Join (Join at
main(HighParallelismIterationsTestProgram.java:60)) -> Combine (MIN(1), at
main(Hig
hParallelismIterationsTestProgram.java:62) (10/25)
(19811ac3816e2e2283596ecdff034321) switched from CREATED to DEPLOYING.
2019-08-12 13:11:48,422 INFO org.apache.flink.runtime.taskmanager.Task
- Creating FileSystem stream leak safety net for task CHAIN Join
(Join at main(HighParallelismIterationsT
estProgram.java:60)) -> Combine (MIN(1), at
main(HighParallelismIterationsTestProgram.java:62) (10/25)
(19811ac3816e2e2283596ecdff034321) [DEPLOYING]
2019-08-12 13:11:48,422 INFO org.apache.flink.runtime.taskmanager.Task
- Loading JAR files for task CHAIN Join (Join at
main(HighParallelismIterationsTestProgram.java:60)) -> Combine (MIN(1), at
main(HighParallelismIterationsTestProgram.java:62) (10/25)
(19811ac3816e2e2283596ecdff034321) [DEPLOYING].
2019-08-12 13:11:50,128 INFO org.apache.flink.runtime.taskmanager.Task
- Attempting to cancel task Partition (25/25)
(32294ea5064d9bb8b67fa7f848909a52).
2019-08-12 13:11:50,128 INFO org.apache.flink.runtime.taskmanager.Task
- Partition (25/25) (32294ea5064d9bb8b67fa7f848909a52) switched
from DEPLOYING to CANCELING.
{noformat}
No other exceptions can be found in the TM logs.
In addition, I have enabled verbose gc logging for TM and JM processes –
nothing suspicious can be found there.
Another problem is that if the Flink test job is not completed successfully,
the test script exits with code 0. We then assert that no exceptions are logged
in the JM and TM logs. It would be cleaner to exit the test script with the
same code that the Flink CLI returns, and maybe skip checking for exceptions in
the logs.
All in all, I would propose the following changes:
- Test should fail if Flink CLI exits with a code other than 0
- Start 2 TMs with 13 slots each instead of 25 TMs with 1 slot
- Make cluster logs part of build artifacts (separate ticket)
> ConnectedComponents end-to-end test instable with NoResourceAvailableException
> ------------------------------------------------------------------------------
>
> Key: FLINK-13484
> URL: https://issues.apache.org/jira/browse/FLINK-13484
> Project: Flink
> Issue Type: Bug
> Components: Test Infrastructure
> Reporter: Tzu-Li (Gordon) Tai
> Assignee: Gary Yao
> Priority: Blocker
> Labels: pull-request-available
> Fix For: 1.9.0
>
> Time Spent: 10m
> Remaining Estimate: 0h
>
> The {{ConnectedComponents iterations with high parallelism}} e2e test seems
> to fail sporadically with {{NoResourceAvailableException}}.
> https://api.travis-ci.org/v3/job/564894454/log.txt
> {code}
> 2019-07-29 18:10:37,294 INFO
> org.apache.flink.runtime.executiongraph.ExecutionGraph - Map (Map at
> main(HighParallelismIterationsTestProgram.java:50)) (9/25)
> (84f306767dabaa104d215bb429797833) switched from SCHEDULED to FAILED.
> org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException:
> Could not allocate enough slots to run the job. Please make sure that the
> cluster has enough resources.
> at
> org.apache.flink.runtime.executiongraph.Execution.lambda$scheduleForExecution$0(Execution.java:459)
> at
> java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
> at
> java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
> at
> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
> at
> java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
> at
> org.apache.flink.runtime.jobmaster.slotpool.SchedulerImpl.lambda$internalAllocateSlot$0(SchedulerImpl.java:190)
> at
> java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
> at
> java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
> at
> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
> at
> java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
> at
> org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$SingleTaskSlot.release(SlotSharingManager.java:694)
> at
> org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$MultiTaskSlot.release(SlotSharingManager.java:482)
> at
> org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$MultiTaskSlot.lambda$new$0(SlotSharingManager.java:378)
> at
> java.util.concurrent.CompletableFuture.uniHandle(CompletableFuture.java:822)
> at
> java.util.concurrent.CompletableFuture$UniHandle.tryFire(CompletableFuture.java:797)
> at
> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
> at
> java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
> at
> org.apache.flink.runtime.concurrent.FutureUtils$Timeout.run(FutureUtils.java:943)
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:397)
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:190)
> at
> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:152)
> at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26)
> at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21)
> at scala.PartialFunction.applyOrElse(PartialFunction.scala:123)
> at scala.PartialFunction.applyOrElse$(PartialFunction.scala:122)
> at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21)
> at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
> at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172)
> at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172)
> at akka.actor.Actor.aroundReceive(Actor.scala:517)
> at akka.actor.Actor.aroundReceive$(Actor.scala:515)
> at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225)
> at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592)
> at akka.actor.ActorCell.invoke(ActorCell.scala:561)
> at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258)
> at akka.dispatch.Mailbox.run(Mailbox.scala:225)
> at akka.dispatch.Mailbox.exec(Mailbox.scala:235)
> at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
> at
> akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
> at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
> at
> akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> 2019-07-29 18:10:37,299 INFO
> org.apache.flink.runtime.executiongraph.failover.AdaptedRestartPipelinedRegionStrategyNG
> - Fail to pass the restart strategy validation in region failover. Fallback
> to fail global.
> 2019-07-29 18:10:37,299 INFO
> org.apache.flink.runtime.executiongraph.ExecutionGraph - Job Flink
> Java Job at Mon Jul 29 18:05:26 UTC 2019 (2fd5c4d1583d85fb81ad98ea1176f9b4)
> switched from state RUNNING to FAILING.
> org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException:
> Could not allocate enough slots to run the job. Please make sure that the
> cluster has enough resources.
> at
> org.apache.flink.runtime.executiongraph.Execution.lambda$scheduleForExecution$0(Execution.java:459)
> at
> java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
> at
> java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
> at
> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
> at
> java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
> at
> org.apache.flink.runtime.jobmaster.slotpool.SchedulerImpl.lambda$internalAllocateSlot$0(SchedulerImpl.java:190)
> at
> java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
> at
> java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
> at
> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
> at
> java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
> at
> org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$SingleTaskSlot.release(SlotSharingManager.java:694)
> at
> org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$MultiTaskSlot.release(SlotSharingManager.java:482)
> at
> org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$MultiTaskSlot.lambda$new$0(SlotSharingManager.java:378)
> at
> java.util.concurrent.CompletableFuture.uniHandle(CompletableFuture.java:822)
> at
> java.util.concurrent.CompletableFuture$UniHandle.tryFire(CompletableFuture.java:797)
> at
> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
> at
> java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
> at
> org.apache.flink.runtime.concurrent.FutureUtils$Timeout.run(FutureUtils.java:943)
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:397)
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:190)
> at
> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
> at
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:152)
> at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26)
> at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21)
> at scala.PartialFunction.applyOrElse(PartialFunction.scala:123)
> at scala.PartialFunction.applyOrElse$(PartialFunction.scala:122)
> at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21)
> at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
> at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172)
> at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:172)
> at akka.actor.Actor.aroundReceive(Actor.scala:517)
> at akka.actor.Actor.aroundReceive$(Actor.scala:515)
> at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225)
> at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592)
> at akka.actor.ActorCell.invoke(ActorCell.scala:561)
> at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258)
> at akka.dispatch.Mailbox.run(Mailbox.scala:225)
> at akka.dispatch.Mailbox.exec(Mailbox.scala:235)
> at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
> at
> akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
> at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
> at
> akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> {code}
--
This message was sent by Atlassian JIRA
(v7.6.14#76016)