From da987138dd1d38ec041e8a5bee7933da2a3771e1 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Tue, 26 Jul 2011 22:23:16 -0600 Subject: [PATCH 01/53] Partial fix for ticket #1054: execute callbacks in dispatcher --- .../test/scala/akka/dispatch/FutureSpec.scala | 21 ++-- .../main/scala/akka/dispatch/Dispatcher.scala | 2 +- .../src/main/scala/akka/dispatch/Future.scala | 95 +++++++++++-------- .../scala/akka/dispatch/MessageHandling.scala | 42 ++++---- .../testkit/CallingThreadDispatcher.scala | 4 +- 5 files changed, 89 insertions(+), 75 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala index f30a331981..c7696b6215 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala @@ -391,7 +391,7 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd val f1 = Future { throw new ThrowableTest("test") } f1.await - intercept[ThrowableTest] { f1.resultOrException } + intercept[ThrowableTest] { f1.get } val latch = new StandardLatch val f2 = Future { latch.tryAwait(5, TimeUnit.SECONDS); "success" } @@ -400,14 +400,17 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd val f3 = f2 map (s ⇒ s.toUpperCase) latch.open f2.await - assert(f2.resultOrException === Some("success")) + assert(f2.get === "success") f2 foreach (_ ⇒ throw new ThrowableTest("current thread foreach")) f2 onResult { case _ ⇒ throw new ThrowableTest("current thread receive") } f3.await - assert(f3.resultOrException === Some("SUCCESS")) + assert(f3.get === "SUCCESS") + + // give time for all callbacks to execute + Thread sleep 100 // make sure all futures are completed in dispatcher - assert(Dispatchers.defaultGlobalDispatcher.pendingFutures === 0) + assert(Dispatchers.defaultGlobalDispatcher.pendingTasks === 0) } "shouldBlockUntilResult" in { @@ -519,7 +522,7 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd Thread.sleep(100) // make sure all futures are completed in dispatcher - assert(Dispatchers.defaultGlobalDispatcher.pendingFutures === 0) + assert(Dispatchers.defaultGlobalDispatcher.pendingTasks === 0) } "shouldNotAddOrRunCallbacksAfterFailureToBeCompletedBeforeExpiry" in { @@ -726,12 +729,12 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd "ticket812FutureDispatchCleanup" in { implicit val dispatcher = new Dispatcher("ticket812FutureDispatchCleanup") - assert(dispatcher.pendingFutures === 0) + assert(dispatcher.pendingTasks === 0) val future = Future({ Thread.sleep(100); "Done" }, 10) intercept[FutureTimeoutException] { future.await } - assert(dispatcher.pendingFutures === 1) - Thread.sleep(100) - assert(dispatcher.pendingFutures === 0) + assert(dispatcher.pendingTasks === 1) + Thread.sleep(200) + assert(dispatcher.pendingTasks === 0) } } } diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala index fbd32c580b..d2b880041c 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala @@ -96,7 +96,7 @@ class Dispatcher( registerForExecution(mbox) } - private[akka] def executeFuture(invocation: FutureInvocation[_]): Unit = if (active.isOn) { + private[akka] def executeTask(invocation: TaskInvocation): Unit = if (active.isOn) { try executorService.get() execute invocation catch { case e: RejectedExecutionException ⇒ diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 547034fd0b..4a378c599a 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -238,8 +238,19 @@ object Future { * This method constructs and returns a Future that will eventually hold the result of the execution of the supplied body * The execution is performed by the specified Dispatcher. */ - def apply[T](body: ⇒ T)(implicit dispatcher: MessageDispatcher, timeout: Timeout = implicitly): Future[T] = - dispatcher.dispatchFuture(() ⇒ body, timeout) + def apply[T](body: ⇒ T)(implicit dispatcher: MessageDispatcher, timeout: Timeout = implicitly): Future[T] = { + val promise = new DefaultPromise[T](timeout) + dispatcher dispatchTask { () ⇒ + promise complete { + try { + Right(body) + } catch { + case e ⇒ Left(e) + } + } + } + promise + } def apply[T](body: ⇒ T, timeout: Timeout)(implicit dispatcher: MessageDispatcher): Future[T] = apply(body)(dispatcher, timeout) @@ -293,9 +304,13 @@ object Future { * * The Delimited Continuations compiler plugin must be enabled in order to use this method. */ - def flow[A](body: ⇒ A @cps[Future[Any]])(implicit timeout: Timeout): Future[A] = { + def flow[A](body: ⇒ A @cps[Future[Any]])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Future[A] = { val future = Promise[A](timeout) - (reset(future.asInstanceOf[Promise[Any]].completeWithResult(body)): Future[Any]) onException { case e ⇒ future completeWithException e } + //dispatcher dispatchTask { () ⇒ + reify(body) foreachFull (future completeWithResult, future completeWithException) onException { + case e: Exception ⇒ future completeWithException e + } + //} future } } @@ -312,7 +327,7 @@ sealed trait Future[+T] { * execution will fail. The normal result of getting a Future from an ActorRef using ? will return * an untyped Future. */ - def apply[A >: T](): A @cps[Future[Any]] = shift(this flatMap (_: A ⇒ Future[Any])) + def apply[A >: T]()(implicit dispatcher: MessageDispatcher, timeout: Timeout): A @cps[Future[Any]] = shift(this flatMap (_: A ⇒ Future[Any])) /** * Blocks awaiting completion of this Future, then returns the resulting value, @@ -407,7 +422,7 @@ sealed trait Future[+T] { * Future. If the Future has already been completed, this will apply * immediately. */ - def onComplete(func: Future[T] ⇒ Unit): this.type + def onComplete(func: Future[T] ⇒ Unit)(implicit dispatcher: MessageDispatcher): this.type /** * When the future is completed with a valid result, apply the provided @@ -419,7 +434,7 @@ sealed trait Future[+T] { * } * */ - final def onResult(pf: PartialFunction[Any, Unit]): this.type = onComplete { f ⇒ + final def onResult(pf: PartialFunction[Any, Unit])(implicit dispatcher: MessageDispatcher): this.type = onComplete { f ⇒ val optr = f.result if (optr.isDefined) { val r = optr.get @@ -437,7 +452,7 @@ sealed trait Future[+T] { * } * */ - final def onException(pf: PartialFunction[Throwable, Unit]): Future[T] = onComplete { f ⇒ + final def onException(pf: PartialFunction[Throwable, Unit])(implicit dispatcher: MessageDispatcher): Future[T] = onComplete { f ⇒ val opte = f.exception if (opte.isDefined) { val e = opte.get @@ -445,9 +460,9 @@ sealed trait Future[+T] { } } - def onTimeout(func: Future[T] ⇒ Unit): this.type + def onTimeout(func: Future[T] ⇒ Unit)(implicit dispatcher: MessageDispatcher): this.type - def orElse[A >: T](fallback: ⇒ A): Future[A] + def orElse[A >: T](fallback: ⇒ A)(implicit dispatcher: MessageDispatcher): Future[A] /** * Creates a new Future by applying a PartialFunction to the successful @@ -463,7 +478,7 @@ sealed trait Future[+T] { * } yield b + "-" + c * */ - final def collect[A](pf: PartialFunction[Any, A])(implicit timeout: Timeout): Future[A] = value match { + final def collect[A](pf: PartialFunction[Any, A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Future[A] = value match { case Some(Right(r)) ⇒ new KeptPromise[A](try { if (pf isDefinedAt r) @@ -509,7 +524,7 @@ sealed trait Future[+T] { * Future(6 / 2) recover { case e: ArithmeticException => 0 } // result: 3 * */ - final def recover[A >: T](pf: PartialFunction[Throwable, A])(implicit timeout: Timeout): Future[A] = value match { + final def recover[A >: T](pf: PartialFunction[Throwable, A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Future[A] = value match { case Some(Left(e)) ⇒ try { if (pf isDefinedAt e) @@ -556,7 +571,7 @@ sealed trait Future[+T] { * } yield b + "-" + c * */ - final def map[A](f: T ⇒ A)(implicit timeout: Timeout): Future[A] = value match { + final def map[A](f: T ⇒ A)(implicit dispatcher: MessageDispatcher, timeout: Timeout): Future[A] = value match { case Some(Right(r)) ⇒ new KeptPromise[A](try { Right(f(r)) @@ -591,7 +606,7 @@ sealed trait Future[+T] { * Creates a new Future[A] which is completed with this Future's result if * that conforms to A's erased type or a ClassCastException otherwise. */ - final def mapTo[A](implicit m: Manifest[A], timeout: Timeout = this.timeout): Future[A] = value match { + final def mapTo[A](implicit m: Manifest[A], dispatcher: MessageDispatcher = implicitly, timeout: Timeout = this.timeout): Future[A] = value match { case Some(Right(t)) ⇒ new KeptPromise(try { Right(BoxedType(m.erasure).cast(t).asInstanceOf[A]) @@ -630,7 +645,7 @@ sealed trait Future[+T] { * } yield b + "-" + c * */ - final def flatMap[A](f: T ⇒ Future[A])(implicit timeout: Timeout): Future[A] = value match { + final def flatMap[A](f: T ⇒ Future[A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Future[A] = value match { case Some(Right(r)) ⇒ try { f(r) @@ -659,23 +674,23 @@ sealed trait Future[+T] { future } - final def foreach(f: T ⇒ Unit): Unit = onComplete { + final def foreach(f: T ⇒ Unit)(implicit dispatcher: MessageDispatcher): Unit = onComplete { _.result match { case Some(v) ⇒ f(v) case None ⇒ } } - final def withFilter(p: T ⇒ Boolean) = new FutureWithFilter[T](this, p) + final def withFilter(p: T ⇒ Boolean)(implicit dispatcher: MessageDispatcher, timeout: Timeout) = new FutureWithFilter[T](this, p) - final class FutureWithFilter[+A](self: Future[A], p: A ⇒ Boolean)(implicit timeout: Timeout) { + final class FutureWithFilter[+A](self: Future[A], p: A ⇒ Boolean)(implicit dispatcher: MessageDispatcher, timeout: Timeout) { def foreach(f: A ⇒ Unit): Unit = self filter p foreach f def map[B](f: A ⇒ B): Future[B] = self filter p map f def flatMap[B](f: A ⇒ Future[B]): Future[B] = self filter p flatMap f def withFilter(q: A ⇒ Boolean): FutureWithFilter[A] = new FutureWithFilter[A](self, x ⇒ p(x) && q(x)) } - final def filter(p: T ⇒ Boolean)(implicit timeout: Timeout): Future[T] = value match { + final def filter(p: T ⇒ Boolean)(implicit dispatcher: MessageDispatcher, timeout: Timeout): Future[T] = value match { case Some(Right(r)) ⇒ try { if (p(r)) @@ -767,26 +782,26 @@ trait Promise[T] extends Future[T] { * Completes this Future with the specified result, if not already completed. * @return this */ - def complete(value: Either[Throwable, T]): this.type + def complete(value: Either[Throwable, T])(implicit dispatcher: MessageDispatcher): this.type /** * Completes this Future with the specified result, if not already completed. * @return this */ - final def completeWithResult(result: T): this.type = complete(Right(result)) + final def completeWithResult(result: T)(implicit dispatcher: MessageDispatcher): this.type = complete(Right(result)) /** * Completes this Future with the specified exception, if not already completed. * @return this */ - final def completeWithException(exception: Throwable): this.type = complete(Left(exception)) + final def completeWithException(exception: Throwable)(implicit dispatcher: MessageDispatcher): this.type = complete(Left(exception)) /** * Completes this Future with the specified other Future, when that Future is completed, * unless this Future has already been completed. * @return this. */ - final def completeWith(other: Future[T]): this.type = { + final def completeWith(other: Future[T])(implicit dispatcher: MessageDispatcher): this.type = { other onComplete { f ⇒ complete(f.value.get) } this } @@ -794,7 +809,7 @@ trait Promise[T] extends Future[T] { final def <<(value: T): Future[T] @cps[Future[Any]] = shift { cont: (Future[T] ⇒ Future[Any]) ⇒ cont(complete(Right(value))) } final def <<(other: Future[T]): Future[T] @cps[Future[Any]] = shift { cont: (Future[T] ⇒ Future[Any]) ⇒ - val fr = new DefaultPromise[Any]() + val fr = new DefaultPromise[Any](this.timeout) this completeWith other onComplete { f ⇒ try { fr completeWith cont(f) @@ -808,7 +823,7 @@ trait Promise[T] extends Future[T] { } final def <<(stream: PromiseStreamOut[T]): Future[T] @cps[Future[Any]] = shift { cont: (Future[T] ⇒ Future[Any]) ⇒ - val fr = Promise[Any]() + val fr = new DefaultPromise[Any](this.timeout) stream.dequeue(this).onComplete { f ⇒ try { fr completeWith cont(f) @@ -892,7 +907,7 @@ class DefaultPromise[T](val timeout: Timeout) extends Promise[T] { } } - def complete(value: Either[Throwable, T]): this.type = { + def complete(value: Either[Throwable, T])(implicit dispatcher: MessageDispatcher): this.type = { _lock.lock val notifyTheseListeners = try { if (_value.isEmpty) { //Only complete if we aren't expired @@ -928,18 +943,20 @@ class DefaultPromise[T](val timeout: Timeout) extends Promise[T] { notifyTheseListeners foreach doNotify }) } else { - try { - val callbacks = Stack[() ⇒ Unit]() // Allocate new aggregator for pending callbacks - Promise.callbacksPendingExecution.set(Some(callbacks)) // Specify the callback aggregator - runCallbacks(notifyTheseListeners, callbacks) // Execute callbacks, if they trigger new callbacks, they are aggregated - } finally { Promise.callbacksPendingExecution.set(None) } // Ensure cleanup + dispatcher dispatchTask { () ⇒ + try { + val callbacks = Stack[() ⇒ Unit]() // Allocate new aggregator for pending callbacks + Promise.callbacksPendingExecution.set(Some(callbacks)) // Specify the callback aggregator + runCallbacks(notifyTheseListeners, callbacks) // Execute callbacks, if they trigger new callbacks, they are aggregated + } finally { Promise.callbacksPendingExecution.set(None) } // Ensure cleanup + } } } this } - def onComplete(func: Future[T] ⇒ Unit): this.type = { + def onComplete(func: Future[T] ⇒ Unit)(implicit dispatcher: MessageDispatcher): this.type = { _lock.lock val notifyNow = try { if (_value.isEmpty) { @@ -952,12 +969,12 @@ class DefaultPromise[T](val timeout: Timeout) extends Promise[T] { _lock.unlock } - if (notifyNow) notifyCompleted(func) + if (notifyNow) dispatcher dispatchTask (() ⇒ notifyCompleted(func)) this } - def onTimeout(func: Future[T] ⇒ Unit): this.type = { + def onTimeout(func: Future[T] ⇒ Unit)(implicit dispatcher: MessageDispatcher): this.type = { if (timeout.duration.isFinite) { _lock.lock val runNow = try { @@ -982,7 +999,7 @@ class DefaultPromise[T](val timeout: Timeout) extends Promise[T] { this } - final def orElse[A >: T](fallback: ⇒ A): Future[A] = + final def orElse[A >: T](fallback: ⇒ A)(implicit dispatcher: MessageDispatcher): Future[A] = if (timeout.duration.isFinite) { value match { case Some(_) ⇒ this @@ -1047,14 +1064,14 @@ object ActorPromise { sealed class KeptPromise[T](suppliedValue: Either[Throwable, T]) extends Promise[T] { val value = Some(suppliedValue) - def complete(value: Either[Throwable, T]): this.type = this - def onComplete(func: Future[T] ⇒ Unit): this.type = { func(this); this } + def complete(value: Either[Throwable, T])(implicit dispatcher: MessageDispatcher): this.type = this + def onComplete(func: Future[T] ⇒ Unit)(implicit dispatcher: MessageDispatcher): this.type = { func(this); this } def await(atMost: Duration): this.type = this def await: this.type = this def isExpired: Boolean = true def timeout: Timeout = Timeout.zero - final def onTimeout(func: Future[T] ⇒ Unit): this.type = this - final def orElse[A >: T](fallback: ⇒ A): Future[A] = this + final def onTimeout(func: Future[T] ⇒ Unit)(implicit dispatcher: MessageDispatcher): this.type = this + final def orElse[A >: T](fallback: ⇒ A)(implicit dispatcher: MessageDispatcher): Future[A] = this } diff --git a/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala b/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala index aafd0988ee..7f4437ed4c 100644 --- a/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala +++ b/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala @@ -26,17 +26,15 @@ final case class MessageInvocation(val receiver: ActorRef, } } -final case class FutureInvocation[T](future: Promise[T], function: () ⇒ T, cleanup: () ⇒ Unit) extends Runnable { +final case class TaskInvocation(function: () ⇒ Unit, cleanup: () ⇒ Unit) extends Runnable { def run() { - future complete (try { - Right(function()) + try { + function() } catch { - case e ⇒ - EventHandler.error(e, this, e.getMessage) - Left(e) + case e ⇒ EventHandler.error(e, this, e.getMessage) } finally { cleanup() - }) + } } } @@ -56,7 +54,7 @@ trait MessageDispatcher { import MessageDispatcher._ protected val uuids = new ConcurrentSkipListSet[Uuid] - protected val futures = new AtomicLong(0L) + protected val tasks = new AtomicLong(0L) protected val guard = new ReentrantGuard protected val active = new Switch(false) @@ -94,31 +92,27 @@ trait MessageDispatcher { dispatch(invocation) } - private[akka] final def dispatchFuture[T](block: () ⇒ T, timeout: Timeout): Future[T] = { - futures.getAndIncrement() + private[akka] final def dispatchTask(block: () ⇒ Unit): Unit = { + tasks.getAndIncrement() try { - val future = new DefaultPromise[T](timeout) - if (active.isOff) guard withGuard { active.switchOn { start() } } - - executeFuture(FutureInvocation[T](future, block, futureCleanup)) - future + executeTask(TaskInvocation(block, taskCleanup)) } catch { case e ⇒ - futures.decrementAndGet + tasks.decrementAndGet throw e } } - private val futureCleanup: () ⇒ Unit = - () ⇒ if (futures.decrementAndGet() == 0) { + private val taskCleanup: () ⇒ Unit = + () ⇒ if (tasks.decrementAndGet() == 0) { guard withGuard { - if (futures.get == 0 && uuids.isEmpty) { + if (tasks.get == 0 && uuids.isEmpty) { shutdownSchedule match { case UNSCHEDULED ⇒ shutdownSchedule = SCHEDULED @@ -155,7 +149,7 @@ trait MessageDispatcher { if (uuids remove actorRef.uuid) { cleanUpMailboxFor(actorRef) actorRef.mailbox = null - if (uuids.isEmpty && futures.get == 0) { + if (uuids.isEmpty && tasks.get == 0) { shutdownSchedule match { case UNSCHEDULED ⇒ shutdownSchedule = SCHEDULED @@ -196,7 +190,7 @@ trait MessageDispatcher { shutdownSchedule = SCHEDULED Scheduler.scheduleOnce(this, timeoutMs, TimeUnit.MILLISECONDS) case SCHEDULED ⇒ - if (uuids.isEmpty && futures.get == 0) { + if (uuids.isEmpty && tasks.get == 0) { active switchOff { shutdown() // shut down in the dispatcher's references is zero } @@ -229,7 +223,7 @@ trait MessageDispatcher { */ private[akka] def dispatch(invocation: MessageInvocation) - private[akka] def executeFuture(invocation: FutureInvocation[_]) + private[akka] def executeTask(invocation: TaskInvocation) /** * Called one time every time an actor is attached to this dispatcher and this dispatcher was previously shutdown @@ -252,9 +246,9 @@ trait MessageDispatcher { def mailboxIsEmpty(actorRef: ActorRef): Boolean /** - * Returns the amount of futures queued for execution + * Returns the amount of tasks queued for execution */ - def pendingFutures: Long = futures.get + def pendingTasks: Long = tasks.get } /** diff --git a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala index 0f0344cc49..fc89966964 100644 --- a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala +++ b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala @@ -5,7 +5,7 @@ package akka.testkit import akka.event.EventHandler import akka.actor.ActorRef -import akka.dispatch.{ MessageDispatcher, MessageInvocation, FutureInvocation, Promise, ActorPromise } +import akka.dispatch.{ MessageDispatcher, MessageInvocation, TaskInvocation, Promise, ActorPromise } import java.util.concurrent.locks.ReentrantLock import java.util.LinkedList import java.util.concurrent.RejectedExecutionException @@ -161,7 +161,7 @@ class CallingThreadDispatcher(val name: String = "calling-thread", val warnings: if (execute) runQueue(mbox, queue) } - private[akka] override def executeFuture(invocation: FutureInvocation[_]) { invocation.run } + private[akka] override def executeTask(invocation: TaskInvocation) { invocation.run } /* * This method must be called with this thread's queue, which must already From 5cb459c25096a1ca1163a07f2801090a9bcc31d0 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Wed, 27 Jul 2011 22:20:02 -0600 Subject: [PATCH 02/53] reverting optimized Future methods due to more consistent behavior and performance increase was small --- .../src/main/scala/akka/dispatch/Future.scala | 265 +++++++----------- 1 file changed, 96 insertions(+), 169 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 4a378c599a..67243914d4 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -478,39 +478,25 @@ sealed trait Future[+T] { * } yield b + "-" + c * */ - final def collect[A](pf: PartialFunction[Any, A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Future[A] = value match { - case Some(Right(r)) ⇒ - new KeptPromise[A](try { - if (pf isDefinedAt r) - Right(pf(r)) - else - Left(new MatchError(r)) - } catch { - case e: Exception ⇒ - EventHandler.error(e, this, e.getMessage) - Left(e) - }) - case Some(_) ⇒ - this.asInstanceOf[Future[A]] - case None ⇒ - val future = new DefaultPromise[A](timeout) - onComplete { self ⇒ - future complete { - self.value.get match { - case Right(r) ⇒ - try { - if (pf isDefinedAt r) Right(pf(r)) - else Left(new MatchError(r)) - } catch { - case e: Exception ⇒ - EventHandler.error(e, this, e.getMessage) - Left(e) - } - case v ⇒ v.asInstanceOf[Either[Throwable, A]] - } + final def collect[A](pf: PartialFunction[Any, A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Future[A] = { + val future = new DefaultPromise[A](timeout) + onComplete { self ⇒ + future complete { + self.value.get match { + case Right(r) ⇒ + try { + if (pf isDefinedAt r) Right(pf(r)) + else Left(new MatchError(r)) + } catch { + case e: Exception ⇒ + EventHandler.error(e, this, e.getMessage) + Left(e) + } + case v ⇒ v.asInstanceOf[Either[Throwable, A]] } } - future + } + future } /** @@ -524,38 +510,24 @@ sealed trait Future[+T] { * Future(6 / 2) recover { case e: ArithmeticException => 0 } // result: 3 * */ - final def recover[A >: T](pf: PartialFunction[Throwable, A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Future[A] = value match { - case Some(Left(e)) ⇒ - try { - if (pf isDefinedAt e) - new KeptPromise(Right(pf(e))) - else - this.asInstanceOf[Future[A]] - } catch { - case e: Exception ⇒ - EventHandler.error(e, this, e.getMessage) - new KeptPromise(Left(e)) - } - case Some(_) ⇒ - this.asInstanceOf[Future[A]] - case None ⇒ - val future = new DefaultPromise[A](timeout) - onComplete { self ⇒ - future complete { - self.value.get match { - case Left(e) ⇒ - try { - if (pf isDefinedAt e) Right(pf(e)) - else Left(e) - } catch { - case x: Exception ⇒ - Left(x) - } - case v ⇒ v - } + final def recover[A >: T](pf: PartialFunction[Throwable, A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Future[A] = { + val future = new DefaultPromise[A](timeout) + onComplete { self ⇒ + future complete { + self.value.get match { + case Left(e) ⇒ + try { + if (pf isDefinedAt e) Right(pf(e)) + else Left(e) + } catch { + case x: Exception ⇒ + Left(x) + } + case v ⇒ v } } - future + } + future } /** @@ -571,64 +543,44 @@ sealed trait Future[+T] { * } yield b + "-" + c * */ - final def map[A](f: T ⇒ A)(implicit dispatcher: MessageDispatcher, timeout: Timeout): Future[A] = value match { - case Some(Right(r)) ⇒ - new KeptPromise[A](try { - Right(f(r)) - } catch { - case e: Exception ⇒ - EventHandler.error(e, this, e.getMessage) - Left(e) - }) - case Some(_) ⇒ - this.asInstanceOf[Future[A]] - case None ⇒ - val future = new DefaultPromise[A](timeout) - onComplete { self ⇒ - future complete { - self.value.get match { - case Right(r) ⇒ - try { - Right(f(r)) - } catch { - case e: Exception ⇒ - EventHandler.error(e, this, e.getMessage) - Left(e) - } - case v ⇒ v.asInstanceOf[Either[Throwable, A]] - } + final def map[A](f: T ⇒ A)(implicit dispatcher: MessageDispatcher, timeout: Timeout): Future[A] = { + val future = new DefaultPromise[A](timeout) + onComplete { self ⇒ + future complete { + self.value.get match { + case Right(r) ⇒ + try { + Right(f(r)) + } catch { + case e: Exception ⇒ + EventHandler.error(e, this, e.getMessage) + Left(e) + } + case v ⇒ v.asInstanceOf[Either[Throwable, A]] } } - future + } + future } /** * Creates a new Future[A] which is completed with this Future's result if * that conforms to A's erased type or a ClassCastException otherwise. */ - final def mapTo[A](implicit m: Manifest[A], dispatcher: MessageDispatcher = implicitly, timeout: Timeout = this.timeout): Future[A] = value match { - case Some(Right(t)) ⇒ - new KeptPromise(try { - Right(BoxedType(m.erasure).cast(t).asInstanceOf[A]) - } catch { - case e: ClassCastException ⇒ Left(e) + final def mapTo[A](implicit m: Manifest[A], dispatcher: MessageDispatcher = implicitly, timeout: Timeout = this.timeout): Future[A] = { + val fa = new DefaultPromise[A](timeout) + onComplete { ft ⇒ + fa complete (ft.value.get match { + case l: Left[_, _] ⇒ l.asInstanceOf[Either[Throwable, A]] + case Right(t) ⇒ + try { + Right(BoxedType(m.erasure).cast(t).asInstanceOf[A]) + } catch { + case e: ClassCastException ⇒ Left(e) + } }) - case Some(_) ⇒ - this.asInstanceOf[Future[A]] - case None ⇒ - val fa = new DefaultPromise[A](timeout) - onComplete { ft ⇒ - fa complete (ft.value.get match { - case l: Left[_, _] ⇒ l.asInstanceOf[Either[Throwable, A]] - case Right(t) ⇒ - try { - Right(BoxedType(m.erasure).cast(t).asInstanceOf[A]) - } catch { - case e: ClassCastException ⇒ Left(e) - } - }) - } - fa + } + fa } /** @@ -645,33 +597,22 @@ sealed trait Future[+T] { * } yield b + "-" + c * */ - final def flatMap[A](f: T ⇒ Future[A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Future[A] = value match { - case Some(Right(r)) ⇒ - try { - f(r) - } catch { - case e: Exception ⇒ - EventHandler.error(e, this, e.getMessage) - new KeptPromise(Left(e)) + final def flatMap[A](f: T ⇒ Future[A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Future[A] = { + val future = new DefaultPromise[A](timeout) + onComplete { + _.value.get match { + case Right(r) ⇒ + try { + future completeWith f(r) + } catch { + case e: Exception ⇒ + EventHandler.error(e, this, e.getMessage) + future complete Left(e) + } + case v ⇒ future complete v.asInstanceOf[Either[Throwable, A]] } - case Some(_) ⇒ - this.asInstanceOf[Future[A]] - case None ⇒ - val future = new DefaultPromise[A](timeout) - onComplete { - _.value.get match { - case Right(r) ⇒ - try { - future completeWith f(r) - } catch { - case e: Exception ⇒ - EventHandler.error(e, this, e.getMessage) - future complete Left(e) - } - case v ⇒ future complete v.asInstanceOf[Either[Throwable, A]] - } - } - future + } + future } final def foreach(f: T ⇒ Unit)(implicit dispatcher: MessageDispatcher): Unit = onComplete { @@ -690,41 +631,27 @@ sealed trait Future[+T] { def withFilter(q: A ⇒ Boolean): FutureWithFilter[A] = new FutureWithFilter[A](self, x ⇒ p(x) && q(x)) } - final def filter(p: T ⇒ Boolean)(implicit dispatcher: MessageDispatcher, timeout: Timeout): Future[T] = value match { - case Some(Right(r)) ⇒ - try { - if (p(r)) - this - else - new KeptPromise(Left(new MatchError(r))) - } catch { - case e: Exception ⇒ - EventHandler.error(e, this, e.getMessage) - new KeptPromise(Left(e)) - } - case Some(_) ⇒ - this - case None ⇒ - val future = new DefaultPromise[T](timeout) - onComplete { self ⇒ - future complete { - self.value.get match { - case Right(r) ⇒ - try { - if (p(r)) - Right(r) - else - Left(new MatchError(r)) - } catch { - case e: Exception ⇒ - EventHandler.error(e, this, e.getMessage) - Left(e) - } - case v ⇒ v - } + final def filter(p: T ⇒ Boolean)(implicit dispatcher: MessageDispatcher, timeout: Timeout): Future[T] = { + val future = new DefaultPromise[T](timeout) + onComplete { self ⇒ + future complete { + self.value.get match { + case Right(r) ⇒ + try { + if (p(r)) + Right(r) + else + Left(new MatchError(r)) + } catch { + case e: Exception ⇒ + EventHandler.error(e, this, e.getMessage) + Left(e) + } + case v ⇒ v } } - future + } + future } /** From cd41daf589aaf7dcbe601c9a4f8c2637775b857f Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Thu, 28 Jul 2011 09:57:45 -0600 Subject: [PATCH 03/53] Future.onComplete now uses threadlocal callback stack to reduce amount of tasks sent to dispatcher --- .../src/main/scala/akka/dispatch/Future.scala | 58 ++++++++++--------- 1 file changed, 31 insertions(+), 27 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 67243914d4..720838e4eb 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -835,31 +835,37 @@ class DefaultPromise[T](val timeout: Timeout) extends Promise[T] { } def complete(value: Either[Throwable, T])(implicit dispatcher: MessageDispatcher): this.type = { - _lock.lock - val notifyTheseListeners = try { - if (_value.isEmpty) { //Only complete if we aren't expired - if (!isExpired) { - _value = Some(value) - val existingListeners = _listeners - _listeners = Nil - existingListeners - } else { - _listeners = Nil - Nil - } - } else Nil - } finally { - _signal.signalAll - _lock.unlock + processCallbacks { + _lock.lock + try { + if (_value.isEmpty) { //Only complete if we aren't expired + if (!isExpired) { + _value = Some(value) + val existingListeners = _listeners + _listeners = Nil + existingListeners + } else { + _listeners = Nil + Nil + } + } else Nil + } finally { + _signal.signalAll + _lock.unlock + } } - if (notifyTheseListeners.nonEmpty) { // Steps to ensure we don't run into a stack-overflow situation + this + } + + private def processCallbacks(callbacks: List[Future[T] ⇒ Unit])(implicit dispatcher: MessageDispatcher): Unit = { + if (callbacks.nonEmpty) { // Steps to ensure we don't run into a stack-overflow situation @tailrec - def runCallbacks(rest: List[Future[T] ⇒ Unit], callbacks: Stack[() ⇒ Unit]) { + def runCallbacks(rest: List[Future[T] ⇒ Unit], callbackStack: Stack[() ⇒ Unit]) { if (rest.nonEmpty) { notifyCompleted(rest.head) - while (callbacks.nonEmpty) { callbacks.pop().apply() } - runCallbacks(rest.tail, callbacks) + while (callbackStack.nonEmpty) { callbackStack.pop().apply() } + runCallbacks(rest.tail, callbackStack) } } @@ -867,20 +873,18 @@ class DefaultPromise[T](val timeout: Timeout) extends Promise[T] { if (pending.isDefined) { //Instead of nesting the calls to the callbacks (leading to stack overflow) pending.get.push(() ⇒ { // Linearize/aggregate callbacks at top level and then execute val doNotify = notifyCompleted _ //Hoist closure to avoid garbage - notifyTheseListeners foreach doNotify + callbacks foreach doNotify }) } else { dispatcher dispatchTask { () ⇒ try { - val callbacks = Stack[() ⇒ Unit]() // Allocate new aggregator for pending callbacks - Promise.callbacksPendingExecution.set(Some(callbacks)) // Specify the callback aggregator - runCallbacks(notifyTheseListeners, callbacks) // Execute callbacks, if they trigger new callbacks, they are aggregated + val callbackStack = Stack[() ⇒ Unit]() // Allocate new aggregator for pending callbacks + Promise.callbacksPendingExecution.set(Some(callbackStack)) // Specify the callback aggregator + runCallbacks(callbacks, callbackStack) // Execute callbacks, if they trigger new callbacks, they are aggregated } finally { Promise.callbacksPendingExecution.set(None) } // Ensure cleanup } } } - - this } def onComplete(func: Future[T] ⇒ Unit)(implicit dispatcher: MessageDispatcher): this.type = { @@ -896,7 +900,7 @@ class DefaultPromise[T](val timeout: Timeout) extends Promise[T] { _lock.unlock } - if (notifyNow) dispatcher dispatchTask (() ⇒ notifyCompleted(func)) + if (notifyNow) processCallbacks(List(func)) this } From 04ba991b63800eed9ac39b9cf48f19b5cf8df9a8 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Thu, 28 Jul 2011 10:10:41 -0600 Subject: [PATCH 04/53] KeptPromise executes callbacks async --- akka-actor/src/main/scala/akka/dispatch/Future.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 720838e4eb..1949040af9 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -996,7 +996,10 @@ sealed class KeptPromise[T](suppliedValue: Either[Throwable, T]) extends Promise val value = Some(suppliedValue) def complete(value: Either[Throwable, T])(implicit dispatcher: MessageDispatcher): this.type = this - def onComplete(func: Future[T] ⇒ Unit)(implicit dispatcher: MessageDispatcher): this.type = { func(this); this } + def onComplete(func: Future[T] ⇒ Unit)(implicit dispatcher: MessageDispatcher): this.type = { + dispatcher dispatchTask (() => func(this)) //TODO: Use pending callback stack + this + } def await(atMost: Duration): this.type = this def await: this.type = this def isExpired: Boolean = true From 0ea10b96b96bcda292b9c008c98953415c515e25 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Thu, 28 Jul 2011 10:34:08 -0600 Subject: [PATCH 05/53] Formatting fix --- akka-actor/src/main/scala/akka/dispatch/Future.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 1949040af9..b3c2b0c1ed 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -997,7 +997,7 @@ sealed class KeptPromise[T](suppliedValue: Either[Throwable, T]) extends Promise def complete(value: Either[Throwable, T])(implicit dispatcher: MessageDispatcher): this.type = this def onComplete(func: Future[T] ⇒ Unit)(implicit dispatcher: MessageDispatcher): this.type = { - dispatcher dispatchTask (() => func(this)) //TODO: Use pending callback stack + dispatcher dispatchTask (() ⇒ func(this)) //TODO: Use pending callback stack this } def await(atMost: Duration): this.type = this From 17b16569cff94d499ea4ffb2457c659b174af81a Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Thu, 28 Jul 2011 11:06:48 -0600 Subject: [PATCH 06/53] Add test for ticket 1054 --- .../test/scala/akka/dispatch/FutureSpec.scala | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala index c7696b6215..f843af91c8 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala @@ -736,6 +736,31 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd Thread.sleep(200) assert(dispatcher.pendingTasks === 0) } + + "run callbacks async" in { + val l1, l2, l3, l4, l5, l6 = new StandardLatch + + val f1 = Future { l1.await; "Hello" } + val f2 = f1 map { s ⇒ l2.await; s.length } + + f1 must not be ('completed) + f2 must not be ('completed) + + l1.open + + f1.await must be('completed) + f2 must not be ('completed) + + val f3 = f1 map { s ⇒ l2.await; s.length * 2 } + + f2 must not be ('completed) + f3 must not be ('completed) + + l2.open + + f2.await must be('completed) + f3.await must be('completed) + } } } From 377fc2b1cf1532442afb1f9850e918232c5c2517 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Thu, 28 Jul 2011 11:15:26 -0600 Subject: [PATCH 07/53] Add test for ticket 1054 --- .../test/scala/akka/dispatch/FutureSpec.scala | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala index f843af91c8..8811f1a7da 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala @@ -760,6 +760,22 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd f2.await must be('completed) f3.await must be('completed) + + val p1 = Promise[String]() + val f4 = p1 map { s ⇒ l3.await; s.length } + + p1 must not be ('completed) + f4 must not be ('completed) + + p1 complete Right("Hello") + + p1 must be('completed) + f4 must not be ('completed) + + l3.open + + f4.await must be('completed) + } } } From 320ee3cb4c2b750fd1f22e005e14d5276105d473 Mon Sep 17 00:00:00 2001 From: Peter Veentjer Date: Tue, 2 Aug 2011 09:52:39 +0300 Subject: [PATCH 08/53] ticket #934 --- .../src/main/scala/akka/actor/Actor.scala | 3 + .../src/main/scala/akka/cluster/Cluster.scala | 1 + .../scala/akka/cluster/ClusterActorRef.scala | 5 + .../BadAddressDirectRoutingMultiJvmSpec.scala | 42 ----- .../FailoverDirectRoutingMultiJvmNode1.conf} | 1 + .../FailoverDirectRoutingMultiJvmNode1.opts} | 0 .../FailoverDirectRoutingMultiJvmNode2.conf | 4 + .../FailoverDirectRoutingMultiJvmNode2.opts} | 0 .../FailoverDirectRoutingMultiJvmSpec.scala | 77 ++++++++++ .../homenode/HomeNode1MultiJvmSpec.scala | 59 +++++++ .../homenode/HomeNodeMultiJvmNode1.conf | 6 + .../homenode/HomeNodeMultiJvmNode1.opts | 0 .../homenode/HomeNodeMultiJvmNode2.conf | 6 + .../homenode/HomeNodeMultiJvmNode2.opts | 0 ...ultiReplicaDirectRoutingMultiJvmNode1.conf | 3 - ...ultiReplicaDirectRoutingMultiJvmSpec.scala | 66 -------- ...ngleReplicaDirectRoutingMultiJvmNode1.conf | 4 + ...ngleReplicaDirectRoutingMultiJvmNode1.opts | 0 ...ngleReplicaDirectRoutingMultiJvmNode2.conf | 4 + ...ngleReplicaDirectRoutingMultiJvmNode2.opts | 0 ...ngleReplicaDirectRoutingMultiJvmSpec.scala | 44 +++--- ...ngleReplicaDirectRoutingMultiJvmNode2.conf | 3 - .../RandomFailoverMultiJvmNode1.conf} | 6 +- .../RandomFailoverMultiJvmNode1.opts} | 0 .../RandomFailoverMultiJvmNode2.conf} | 6 +- .../RandomFailoverMultiJvmNode2.opts} | 0 .../failover/RandomFailoverMultiJvmNode3.conf | 5 + .../RandomFailoverMultiJvmNode3.opts} | 0 .../failover/RandomFailoverMultiJvmSpec.scala | 119 +++++++++++++++ .../homenode/HomeNodeMultiJvmNode1.conf | 8 + .../homenode/HomeNodeMultiJvmNode1.opts} | 0 .../homenode/HomeNodeMultiJvmNode2.conf | 8 + .../homenode/HomeNodeMultiJvmNode2.opts} | 0 .../homenode/HomeNodeMultiJvmSpec.scala | 59 +++++++ .../Random1ReplicaMultiJvmNode1.conf} | 2 +- .../Random1ReplicaMultiJvmNode1.opts} | 0 .../Random1ReplicaMultiJvmSpec.scala | 50 ++++++ .../Random3ReplicasMultiJvmNode1.conf} | 5 +- .../Random3ReplicasMultiJvmNode1.opts} | 0 .../Random3ReplicasMultiJvmNode2.conf | 4 + .../Random3ReplicasMultiJvmNode2.opts} | 0 .../Random3ReplicasMultiJvmNode3.conf | 4 + .../Random3ReplicasMultiJvmNode3.opts} | 0 .../Random3ReplicasMultiJvmSpec.scala | 111 ++++++++++++++ .../RoundRobinFailoverMultiJvmNode1.conf | 4 +- .../RoundRobinFailoverMultiJvmNode1.opts | 0 .../RoundRobinFailoverMultiJvmNode2.conf | 4 +- .../RoundRobinFailoverMultiJvmNode2.opts | 1 + .../RoundRobinFailoverMultiJvmNode3.conf | 4 +- .../RoundRobinFailoverMultiJvmNode3.opts | 1 + .../RoundRobinFailoverMultiJvmSpec.scala | 120 +++++++++++++++ .../homenode/HomeNodeMultiJvmNode1.conf | 8 + .../homenode/HomeNodeMultiJvmNode1.opts | 1 + .../homenode/HomeNodeMultiJvmNode2.conf | 0 .../homenode/HomeNodeMultiJvmNode2.opts | 1 + .../homenode/HomeNodeMultiJvmSpec.scala | 27 ++-- .../RoundRobin1ReplicaMultiJvmNode1.conf | 0 .../RoundRobin1ReplicaMultiJvmNode1.opts | 1 + .../RoundRobin1ReplicaMultiJvmSpec.scala | 7 +- .../RoundRobin2ReplicasMultiJvmNode1.conf | 0 .../RoundRobin2ReplicasMultiJvmNode1.opts | 1 + .../RoundRobin2ReplicasMultiJvmNode2.conf | 0 .../RoundRobin2ReplicasMultiJvmNode2.opts | 1 + .../RoundRobin2ReplicasMultiJvmSpec.scala | 18 +-- .../RoundRobin3ReplicasMultiJvmNode1.conf | 0 .../RoundRobin3ReplicasMultiJvmNode1.opts | 1 + .../RoundRobin3ReplicasMultiJvmNode2.conf | 0 .../RoundRobin3ReplicasMultiJvmNode2.opts | 1 + .../RoundRobin3ReplicasMultiJvmNode3.conf | 0 .../RoundRobin3ReplicasMultiJvmNode3.opts | 1 + .../RoundRobin3ReplicasMultiJvmSpec.scala | 28 ++-- .../RoundRobinFailoverMultiJvmNode4.opts | 1 - .../RoundRobinFailoverMultiJvmSpec.scala | 144 ------------------ .../routing/roundrobin_failover/questions.txt | 6 - .../testing-design-improvements.txt | 54 ------- 75 files changed, 755 insertions(+), 395 deletions(-) delete mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/bad_address/BadAddressDirectRoutingMultiJvmSpec.scala rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/{multiple_replicas/MultiReplicaDirectRoutingMultiJvmNode2.conf => failover/FailoverDirectRoutingMultiJvmNode1.conf} (65%) rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/{bad_address/BadAddressDirectRoutingMultiJvmNode1.opts => failover/FailoverDirectRoutingMultiJvmNode1.opts} (100%) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/failover/FailoverDirectRoutingMultiJvmNode2.conf rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/{multiple_replicas/MultiReplicaDirectRoutingMultiJvmNode2.opts => failover/FailoverDirectRoutingMultiJvmNode2.opts} (100%) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/failover/FailoverDirectRoutingMultiJvmSpec.scala create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/homenode/HomeNode1MultiJvmSpec.scala create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/homenode/HomeNodeMultiJvmNode1.conf rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{ => direct}/homenode/HomeNodeMultiJvmNode1.opts (100%) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/homenode/HomeNodeMultiJvmNode2.conf rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{ => direct}/homenode/HomeNodeMultiJvmNode2.opts (100%) delete mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/multiple_replicas/MultiReplicaDirectRoutingMultiJvmNode1.conf delete mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/multiple_replicas/MultiReplicaDirectRoutingMultiJvmSpec.scala create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/normalusage/SingleReplicaDirectRoutingMultiJvmNode1.conf rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/{single_replica => normalusage}/SingleReplicaDirectRoutingMultiJvmNode1.opts (100%) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/normalusage/SingleReplicaDirectRoutingMultiJvmNode2.conf rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/{single_replica => normalusage}/SingleReplicaDirectRoutingMultiJvmNode2.opts (100%) rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/{single_replica => normalusage}/SingleReplicaDirectRoutingMultiJvmSpec.scala (82%) delete mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/single_replica/SingleReplicaDirectRoutingMultiJvmNode2.conf rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{homenode/HomeNodeMultiJvmNode1.conf => random/failover/RandomFailoverMultiJvmNode1.conf} (71%) rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{direct/multiple_replicas/MultiReplicaDirectRoutingMultiJvmNode1.opts => random/failover/RandomFailoverMultiJvmNode1.opts} (100%) rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{roundrobin_failover/RoundRobinFailoverMultiJvmNode4.conf => random/failover/RandomFailoverMultiJvmNode2.conf} (71%) rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{roundrobin_2_replicas/RoundRobin2ReplicasMultiJvmNode2.opts => random/failover/RandomFailoverMultiJvmNode2.opts} (100%) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmNode3.conf rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{roundrobin_3_replicas/RoundRobin3ReplicasMultiJvmNode3.opts => random/failover/RandomFailoverMultiJvmNode3.opts} (100%) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmSpec.scala create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/homenode/HomeNodeMultiJvmNode1.conf rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{roundrobin_1_replica/RoundRobin1ReplicaMultiJvmNode1.opts => random/homenode/HomeNodeMultiJvmNode1.opts} (100%) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/homenode/HomeNodeMultiJvmNode2.conf rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{roundrobin_3_replicas/RoundRobin3ReplicasMultiJvmNode2.opts => random/homenode/HomeNodeMultiJvmNode2.opts} (100%) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/homenode/HomeNodeMultiJvmSpec.scala rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{direct/single_replica/SingleReplicaDirectRoutingMultiJvmNode1.conf => random/replicationfactor_1/Random1ReplicaMultiJvmNode1.conf} (72%) rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{roundrobin_2_replicas/RoundRobin2ReplicasMultiJvmNode1.opts => random/replicationfactor_1/Random1ReplicaMultiJvmNode1.opts} (100%) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_1/Random1ReplicaMultiJvmSpec.scala rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{direct/bad_address/BadAddressDirectRoutingMultiJvmNode1.conf => random/replicationfactor_3/Random3ReplicasMultiJvmNode1.conf} (51%) rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{roundrobin_3_replicas/RoundRobin3ReplicasMultiJvmNode1.opts => random/replicationfactor_3/Random3ReplicasMultiJvmNode1.opts} (100%) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmNode2.conf rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{roundrobin_failover/RoundRobinFailoverMultiJvmNode2.opts => random/replicationfactor_3/Random3ReplicasMultiJvmNode2.opts} (100%) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmNode3.conf rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{roundrobin_failover/RoundRobinFailoverMultiJvmNode3.opts => random/replicationfactor_3/Random3ReplicasMultiJvmNode3.opts} (100%) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmSpec.scala rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{roundrobin_failover => roundrobin/failover}/RoundRobinFailoverMultiJvmNode1.conf (93%) rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{roundrobin_failover => roundrobin/failover}/RoundRobinFailoverMultiJvmNode1.opts (100%) rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{roundrobin_failover => roundrobin/failover}/RoundRobinFailoverMultiJvmNode2.conf (93%) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmNode2.opts rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{roundrobin_failover => roundrobin/failover}/RoundRobinFailoverMultiJvmNode3.conf (93%) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmNode3.opts create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmSpec.scala create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/homenode/HomeNodeMultiJvmNode1.conf create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/homenode/HomeNodeMultiJvmNode1.opts rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{ => roundrobin}/homenode/HomeNodeMultiJvmNode2.conf (100%) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/homenode/HomeNodeMultiJvmNode2.opts rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{ => roundrobin}/homenode/HomeNodeMultiJvmSpec.scala (59%) rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{roundrobin_1_replica => roundrobin/replicationfactor_1}/RoundRobin1ReplicaMultiJvmNode1.conf (100%) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_1/RoundRobin1ReplicaMultiJvmNode1.opts rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{roundrobin_1_replica => roundrobin/replicationfactor_1}/RoundRobin1ReplicaMultiJvmSpec.scala (88%) rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{roundrobin_2_replicas => roundrobin/replicationfactor_2}/RoundRobin2ReplicasMultiJvmNode1.conf (100%) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_2/RoundRobin2ReplicasMultiJvmNode1.opts rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{roundrobin_2_replicas => roundrobin/replicationfactor_2}/RoundRobin2ReplicasMultiJvmNode2.conf (100%) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_2/RoundRobin2ReplicasMultiJvmNode2.opts rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{roundrobin_2_replicas => roundrobin/replicationfactor_2}/RoundRobin2ReplicasMultiJvmSpec.scala (90%) rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{roundrobin_3_replicas => roundrobin/replicationfactor_3}/RoundRobin3ReplicasMultiJvmNode1.conf (100%) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmNode1.opts rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{roundrobin_3_replicas => roundrobin/replicationfactor_3}/RoundRobin3ReplicasMultiJvmNode2.conf (100%) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmNode2.opts rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{roundrobin_3_replicas => roundrobin/replicationfactor_3}/RoundRobin3ReplicasMultiJvmNode3.conf (100%) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmNode3.opts rename akka-cluster/src/multi-jvm/scala/akka/cluster/routing/{roundrobin_3_replicas => roundrobin/replicationfactor_3}/RoundRobin3ReplicasMultiJvmSpec.scala (86%) delete mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmNode4.opts delete mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmSpec.scala delete mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/questions.txt delete mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/testing-design-improvements.txt diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index 3a5f906d89..7fec89b4c2 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -108,6 +108,9 @@ class InvalidMessageException private[akka] (message: String, cause: Throwable = * This message is thrown by default when an Actors behavior doesn't match a message */ case class UnhandledMessageException(msg: Any, ref: ActorRef = null) extends Exception { + + def this(msg: String) = this(msg, null) + // constructor with 'null' ActorRef needed to work with client instantiation of remote exception override def getMessage = if (ref ne null) "Actor %s does not handle [%s]".format(ref, msg) diff --git a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala index 348b11195e..37737280ae 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala @@ -1430,6 +1430,7 @@ class DefaultClusterNode private[akka] ( } private[cluster] def failOverClusterActorRefConnections(from: InetSocketAddress, to: InetSocketAddress) { + EventHandler.info(this, "failOverClusterActorRef from %s to %s".format(from, to)) clusterActorRefs.values(from) foreach (_.failOver(from, to)) } diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala index 87b120cf30..a7f8aff909 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala @@ -15,6 +15,7 @@ import com.eaio.uuid.UUID import collection.immutable.Map import annotation.tailrec import akka.routing.Router +import akka.event.EventHandler /** * ActorRef representing a one or many instances of a clustered, load-balanced and sometimes replicated actor @@ -57,6 +58,8 @@ class ClusterActorRef private[akka] (inetSocketAddresses: Array[Tuple2[UUID, Ine } private[akka] def failOver(from: InetSocketAddress, to: InetSocketAddress): Unit = { + EventHandler.info(this, "ClusterActorRef. %s failover from %s to %s".format(address, from, to)) + @tailrec def doFailover(from: InetSocketAddress, to: InetSocketAddress): Unit = { val oldValue = inetSocketAddressToActorRefMap.get @@ -93,6 +96,8 @@ class ClusterActorRef private[akka] (inetSocketAddresses: Array[Tuple2[UUID, Ine } def signalDeadActor(ref: ActorRef): Unit = { + EventHandler.info(this, "ClusterActorRef %s signalDeadActor %s".format(address, ref.address)) + //since the number remote actor refs for a clustered actor ref is quite low, we can deal with the O(N) complexity //of the following removal. val map = inetSocketAddressToActorRefMap.get diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/bad_address/BadAddressDirectRoutingMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/bad_address/BadAddressDirectRoutingMultiJvmSpec.scala deleted file mode 100644 index 75bc03309d..0000000000 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/bad_address/BadAddressDirectRoutingMultiJvmSpec.scala +++ /dev/null @@ -1,42 +0,0 @@ -package akka.cluster.routing.direct.bad_address - -import akka.cluster.{ Cluster, MasterClusterTestNode } -import akka.actor.Actor -import akka.config.Config - -object BadAddressDirectRoutingMultiJvmSpec { - - val NrOfNodes = 1 - - class SomeActor extends Actor with Serializable { - println("---------------------------------------------------------------------------") - println("SomeActor has been created on node [" + Config.nodename + "]") - println("---------------------------------------------------------------------------") - - def receive = { - case "identify" ⇒ { - println("The node received the 'identify' command: " + Config.nodename) - self.reply(Config.nodename) - } - } - } - -} - -class BadAddressDirectRoutingMultiJvmNode1 extends MasterClusterTestNode { - - import BadAddressDirectRoutingMultiJvmSpec._ - - val testNodes = NrOfNodes - - "node" must { - "participate in cluster" in { - Cluster.node.start() - - Cluster.barrier("waiting-for-begin", NrOfNodes).await() - Cluster.barrier("waiting-to-end", NrOfNodes).await() - Cluster.node.shutdown() - } - } -} - diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/multiple_replicas/MultiReplicaDirectRoutingMultiJvmNode2.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/failover/FailoverDirectRoutingMultiJvmNode1.conf similarity index 65% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/multiple_replicas/MultiReplicaDirectRoutingMultiJvmNode2.conf rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/failover/FailoverDirectRoutingMultiJvmNode1.conf index b60f6a3b5c..602bc41489 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/multiple_replicas/MultiReplicaDirectRoutingMultiJvmNode2.conf +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/failover/FailoverDirectRoutingMultiJvmNode1.conf @@ -1,3 +1,4 @@ akka.enabled-modules = ["cluster"] akka.event-handler-level = "WARNING" akka.actor.deployment.service-hello.router = "direct" +akka.actor.deployment.service-hello.clustered.home = "node:node2" \ No newline at end of file diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/bad_address/BadAddressDirectRoutingMultiJvmNode1.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/failover/FailoverDirectRoutingMultiJvmNode1.opts similarity index 100% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/bad_address/BadAddressDirectRoutingMultiJvmNode1.opts rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/failover/FailoverDirectRoutingMultiJvmNode1.opts diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/failover/FailoverDirectRoutingMultiJvmNode2.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/failover/FailoverDirectRoutingMultiJvmNode2.conf new file mode 100644 index 0000000000..602bc41489 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/failover/FailoverDirectRoutingMultiJvmNode2.conf @@ -0,0 +1,4 @@ +akka.enabled-modules = ["cluster"] +akka.event-handler-level = "WARNING" +akka.actor.deployment.service-hello.router = "direct" +akka.actor.deployment.service-hello.clustered.home = "node:node2" \ No newline at end of file diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/multiple_replicas/MultiReplicaDirectRoutingMultiJvmNode2.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/failover/FailoverDirectRoutingMultiJvmNode2.opts similarity index 100% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/multiple_replicas/MultiReplicaDirectRoutingMultiJvmNode2.opts rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/failover/FailoverDirectRoutingMultiJvmNode2.opts diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/failover/FailoverDirectRoutingMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/failover/FailoverDirectRoutingMultiJvmSpec.scala new file mode 100644 index 0000000000..df41e93fd9 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/failover/FailoverDirectRoutingMultiJvmSpec.scala @@ -0,0 +1,77 @@ +package akka.cluster.routing.direct.failover + +import akka.config.Config +import scala.Predef._ +import akka.cluster.{ ClusterActorRef, Cluster, MasterClusterTestNode, ClusterTestNode } +import akka.actor.{ ActorInitializationException, Actor } + +object FailoverDirectRoutingMultiJvmSpec { + + val NrOfNodes = 2 + + class SomeActor extends Actor with Serializable { + println("---------------------------------------------------------------------------") + println("SomeActor has been created on node [" + Config.nodename + "]") + println("---------------------------------------------------------------------------") + + def receive = { + case "identify" ⇒ + println("The node received the 'identify' command: " + Config.nodename) + self.reply(Config.nodename) + case "die" ⇒ + println("The node received the 'die' command: " + Config.nodename) + Cluster.node.shutdown + } + } + +} + +class FailoverDirectRoutingMultiJvmNode1 extends MasterClusterTestNode { + + import FailoverDirectRoutingMultiJvmSpec._ + + val testNodes = NrOfNodes + + "Direct Router" must { + "not yet be able to failover to another node" in { + + println("==================================================================================================") + println(" FAILOVER DIRECT ROUTING") + println("==================================================================================================") + + Cluster.node.start() + + Cluster.barrier("waiting-for-begin", NrOfNodes).await() + val actor = Actor.actorOf[SomeActor]("service-hello").start().asInstanceOf[ClusterActorRef] + + println("retrieved identity was: " + (actor ? "identify").get) + (actor ? "identify").get must equal("node2") + + actor ! "die" + + Thread.sleep(4000) + + try { + actor ! "identify" + fail() + } catch { + case e: ActorInitializationException ⇒ + } + } + } +} + +class FailoverDirectRoutingMultiJvmNode2 extends ClusterTestNode { + + import FailoverDirectRoutingMultiJvmSpec._ + + "___" must { + "___" in { + Cluster.node.start() + Cluster.barrier("waiting-for-begin", NrOfNodes).await() + + Thread.sleep(30 * 1000) + } + } +} + diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/homenode/HomeNode1MultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/homenode/HomeNode1MultiJvmSpec.scala new file mode 100644 index 0000000000..1a8805c337 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/homenode/HomeNode1MultiJvmSpec.scala @@ -0,0 +1,59 @@ +package akka.cluster.routing.direct.homenode + +import akka.config.Config +import akka.actor.Actor +import akka.cluster.{ ClusterTestNode, MasterClusterTestNode, Cluster } +import Cluster._ + +object HomeNodeMultiJvmSpec { + + val NrOfNodes = 2 + + class SomeActor extends Actor with Serializable { + def receive = { + case "identify" ⇒ { + self.reply(Config.nodename) + } + } + } + +} + +class HomeNodeMultiJvmNode1 extends MasterClusterTestNode { + + import HomeNodeMultiJvmSpec._ + + val testNodes = NrOfNodes + + "___" must { + "___" in { + node.start() + barrier("waiting-for-begin", NrOfNodes).await() + barrier("waiting-for-end", NrOfNodes).await() + node.shutdown() + } + } +} + +class HomeNodeMultiJvmNode2 extends ClusterTestNode { + + import HomeNodeMultiJvmSpec._ + + "Direct Router: A Direct Router" must { + "obey 'home-node' config option when instantiated actor in cluster" in { + node.start() + barrier("waiting-for-begin", NrOfNodes).await() + + val actorNode1 = Actor.actorOf[SomeActor]("service-node1").start() + val name1 = (actorNode1 ? "identify").get.asInstanceOf[String] + name1 must equal("node1") + + val actorNode2 = Actor.actorOf[SomeActor]("service-node2").start() + val name2 = (actorNode2 ? "identify").get.asInstanceOf[String] + name2 must equal("node2") + + barrier("waiting-for-end", NrOfNodes).await() + node.shutdown() + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/homenode/HomeNodeMultiJvmNode1.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/homenode/HomeNodeMultiJvmNode1.conf new file mode 100644 index 0000000000..cfe103b985 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/homenode/HomeNodeMultiJvmNode1.conf @@ -0,0 +1,6 @@ +akka.enabled-modules = ["cluster"] +akka.event-handler-level = "WARNING" +akka.actor.deployment.service-node1.router = "direct" +akka.actor.deployment.service-node1.clustered.preferred-nodes = ["node:node1"] +akka.actor.deployment.service-node2.router = "direct" +akka.actor.deployment.service-node2.clustered.preferred-nodes = ["node:node2"] diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/homenode/HomeNodeMultiJvmNode1.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/homenode/HomeNodeMultiJvmNode1.opts similarity index 100% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/homenode/HomeNodeMultiJvmNode1.opts rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/homenode/HomeNodeMultiJvmNode1.opts diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/homenode/HomeNodeMultiJvmNode2.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/homenode/HomeNodeMultiJvmNode2.conf new file mode 100644 index 0000000000..cfe103b985 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/homenode/HomeNodeMultiJvmNode2.conf @@ -0,0 +1,6 @@ +akka.enabled-modules = ["cluster"] +akka.event-handler-level = "WARNING" +akka.actor.deployment.service-node1.router = "direct" +akka.actor.deployment.service-node1.clustered.preferred-nodes = ["node:node1"] +akka.actor.deployment.service-node2.router = "direct" +akka.actor.deployment.service-node2.clustered.preferred-nodes = ["node:node2"] diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/homenode/HomeNodeMultiJvmNode2.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/homenode/HomeNodeMultiJvmNode2.opts similarity index 100% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/homenode/HomeNodeMultiJvmNode2.opts rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/homenode/HomeNodeMultiJvmNode2.opts diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/multiple_replicas/MultiReplicaDirectRoutingMultiJvmNode1.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/multiple_replicas/MultiReplicaDirectRoutingMultiJvmNode1.conf deleted file mode 100644 index 40fcfa5d51..0000000000 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/multiple_replicas/MultiReplicaDirectRoutingMultiJvmNode1.conf +++ /dev/null @@ -1,3 +0,0 @@ -akka.enabled-modules = ["cluster"] -akka.event-handler-level = "WARNING" -akka.actor.deployment.service-hello.router = "direct" \ No newline at end of file diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/multiple_replicas/MultiReplicaDirectRoutingMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/multiple_replicas/MultiReplicaDirectRoutingMultiJvmSpec.scala deleted file mode 100644 index ca1f87503b..0000000000 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/multiple_replicas/MultiReplicaDirectRoutingMultiJvmSpec.scala +++ /dev/null @@ -1,66 +0,0 @@ -package akka.cluster.routing.direct.multiple_replicas - -import akka.actor.Actor -import akka.cluster.{ MasterClusterTestNode, Cluster, ClusterTestNode } -import akka.config.Config - -object MultiReplicaDirectRoutingMultiJvmSpec { - val NrOfNodes = 2 - - class SomeActor extends Actor with Serializable { - println("---------------------------------------------------------------------------") - println("SomeActor has been created on node [" + Config.nodename + "]") - println("---------------------------------------------------------------------------") - - def receive = { - case "identify" ⇒ { - println("The node received the 'identify' command: " + Config.nodename) - self.reply(Config.nodename) - } - } - } - -} - -class MultiReplicaDirectRoutingMultiJvmNode2 extends ClusterTestNode { - - import MultiReplicaDirectRoutingMultiJvmSpec._ - - "when node send message to existing node using direct routing it" must { - "communicate with that node" in { - Cluster.node.start() - Cluster.barrier("waiting-for-begin", NrOfNodes).await() - - //Cluster.barrier("get-ref-to-actor-on-node2", NrOfNodes).await() - - val actor = Actor.actorOf[SomeActor]("service-hello") - actor.start() - - //actor.start() - val name: String = (actor ? "identify").get.asInstanceOf[String] - - println("The name of the actor was " + name) - - Cluster.barrier("waiting-to-end", NrOfNodes).await() - Cluster.node.shutdown() - } - } -} - -class MultiReplicaDirectRoutingMultiJvmNode1 extends MasterClusterTestNode { - - import MultiReplicaDirectRoutingMultiJvmSpec._ - - val testNodes = NrOfNodes - - "node" must { - "participate in cluster" in { - Cluster.node.start() - - Cluster.barrier("waiting-for-begin", NrOfNodes).await() - Cluster.barrier("waiting-to-end", NrOfNodes).await() - Cluster.node.shutdown() - } - } -} - diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/normalusage/SingleReplicaDirectRoutingMultiJvmNode1.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/normalusage/SingleReplicaDirectRoutingMultiJvmNode1.conf new file mode 100644 index 0000000000..e6de1b42f8 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/normalusage/SingleReplicaDirectRoutingMultiJvmNode1.conf @@ -0,0 +1,4 @@ +akka.enabled-modules = ["cluster"] +akka.event-handler-level = "WARNING" +akka.actor.deployment.service-hello.router = "direct" +akka.actor.deployment.service-hello.clustered.preferred-nodes = ["node:node1"] diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/single_replica/SingleReplicaDirectRoutingMultiJvmNode1.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/normalusage/SingleReplicaDirectRoutingMultiJvmNode1.opts similarity index 100% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/single_replica/SingleReplicaDirectRoutingMultiJvmNode1.opts rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/normalusage/SingleReplicaDirectRoutingMultiJvmNode1.opts diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/normalusage/SingleReplicaDirectRoutingMultiJvmNode2.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/normalusage/SingleReplicaDirectRoutingMultiJvmNode2.conf new file mode 100644 index 0000000000..e6de1b42f8 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/normalusage/SingleReplicaDirectRoutingMultiJvmNode2.conf @@ -0,0 +1,4 @@ +akka.enabled-modules = ["cluster"] +akka.event-handler-level = "WARNING" +akka.actor.deployment.service-hello.router = "direct" +akka.actor.deployment.service-hello.clustered.preferred-nodes = ["node:node1"] diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/single_replica/SingleReplicaDirectRoutingMultiJvmNode2.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/normalusage/SingleReplicaDirectRoutingMultiJvmNode2.opts similarity index 100% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/single_replica/SingleReplicaDirectRoutingMultiJvmNode2.opts rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/normalusage/SingleReplicaDirectRoutingMultiJvmNode2.opts diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/single_replica/SingleReplicaDirectRoutingMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/normalusage/SingleReplicaDirectRoutingMultiJvmSpec.scala similarity index 82% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/single_replica/SingleReplicaDirectRoutingMultiJvmSpec.scala rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/normalusage/SingleReplicaDirectRoutingMultiJvmSpec.scala index 35009b6d47..9bc9681263 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/single_replica/SingleReplicaDirectRoutingMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/normalusage/SingleReplicaDirectRoutingMultiJvmSpec.scala @@ -1,8 +1,8 @@ -package akka.cluster.routing.direct.single_replica +package akka.cluster.routing.direct.normalusage import akka.actor.Actor import akka.config.Config -import akka.cluster.{ ClusterTestNode, MasterClusterTestNode, Cluster } +import akka.cluster.{ ClusterActorRef, ClusterTestNode, MasterClusterTestNode, Cluster } object SingleReplicaDirectRoutingMultiJvmSpec { val NrOfNodes = 2 @@ -19,7 +19,6 @@ object SingleReplicaDirectRoutingMultiJvmSpec { } } } - } class SingleReplicaDirectRoutingMultiJvmNode1 extends MasterClusterTestNode { @@ -28,24 +27,6 @@ class SingleReplicaDirectRoutingMultiJvmNode1 extends MasterClusterTestNode { val testNodes = NrOfNodes - "when node send message to existing node using direct routing it" must { - "communicate with that node" in { - Cluster.node.start() - Cluster.barrier("waiting-for-begin", NrOfNodes).await() - - val actor = Actor.actorOf[SomeActor]("service-hello").start() - actor.isRunning must be(true) - - Cluster.barrier("waiting-to-end", NrOfNodes).await() - Cluster.node.shutdown() - } - } -} - -class SingleReplicaDirectRoutingMultiJvmNode2 extends ClusterTestNode { - - import SingleReplicaDirectRoutingMultiJvmSpec._ - "___" must { "___" in { Cluster.node.start() @@ -57,3 +38,24 @@ class SingleReplicaDirectRoutingMultiJvmNode2 extends ClusterTestNode { } } +class SingleReplicaDirectRoutingMultiJvmNode2 extends ClusterTestNode { + + import SingleReplicaDirectRoutingMultiJvmSpec._ + + "Direct Router: when node send message to existing node it" must { + "communicate with that node" in { + Cluster.node.start() + Cluster.barrier("waiting-for-begin", NrOfNodes).await() + + val actor = Actor.actorOf[SomeActor]("service-hello").start().asInstanceOf[ClusterActorRef] + actor.isRunning must be(true) + + val result = (actor ? "identify").get + result must equal("node1") + + Cluster.barrier("waiting-to-end", NrOfNodes).await() + Cluster.node.shutdown() + } + } +} + diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/single_replica/SingleReplicaDirectRoutingMultiJvmNode2.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/single_replica/SingleReplicaDirectRoutingMultiJvmNode2.conf deleted file mode 100644 index 40fcfa5d51..0000000000 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/single_replica/SingleReplicaDirectRoutingMultiJvmNode2.conf +++ /dev/null @@ -1,3 +0,0 @@ -akka.enabled-modules = ["cluster"] -akka.event-handler-level = "WARNING" -akka.actor.deployment.service-hello.router = "direct" \ No newline at end of file diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/homenode/HomeNodeMultiJvmNode1.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmNode1.conf similarity index 71% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/homenode/HomeNodeMultiJvmNode1.conf rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmNode1.conf index 366cf9111c..c75c38be44 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/homenode/HomeNodeMultiJvmNode1.conf +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmNode1.conf @@ -1,5 +1,5 @@ akka.enabled-modules = ["cluster"] akka.event-handler-level = "WARNING" -akka.actor.deployment.service-hello.router = "round-robin" -akka.actor.deployment.service-hello.clustered.preferred-nodes = ["node:node1"] -akka.actor.deployment.service-hello.clustered.replication-factor = 1 \ No newline at end of file +akka.actor.deployment.service-hello.router = "random" +akka.actor.deployment.service-hello.clustered.preferred-nodes = ["node:node2"] +akka.actor.deployment.service-hello.clustered.replication-factor = 2 \ No newline at end of file diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/multiple_replicas/MultiReplicaDirectRoutingMultiJvmNode1.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmNode1.opts similarity index 100% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/multiple_replicas/MultiReplicaDirectRoutingMultiJvmNode1.opts rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmNode1.opts diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmNode4.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmNode2.conf similarity index 71% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmNode4.conf rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmNode2.conf index 366cf9111c..c75c38be44 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmNode4.conf +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmNode2.conf @@ -1,5 +1,5 @@ akka.enabled-modules = ["cluster"] akka.event-handler-level = "WARNING" -akka.actor.deployment.service-hello.router = "round-robin" -akka.actor.deployment.service-hello.clustered.preferred-nodes = ["node:node1"] -akka.actor.deployment.service-hello.clustered.replication-factor = 1 \ No newline at end of file +akka.actor.deployment.service-hello.router = "random" +akka.actor.deployment.service-hello.clustered.preferred-nodes = ["node:node2"] +akka.actor.deployment.service-hello.clustered.replication-factor = 2 \ No newline at end of file diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_2_replicas/RoundRobin2ReplicasMultiJvmNode2.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmNode2.opts similarity index 100% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_2_replicas/RoundRobin2ReplicasMultiJvmNode2.opts rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmNode2.opts diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmNode3.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmNode3.conf new file mode 100644 index 0000000000..c75c38be44 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmNode3.conf @@ -0,0 +1,5 @@ +akka.enabled-modules = ["cluster"] +akka.event-handler-level = "WARNING" +akka.actor.deployment.service-hello.router = "random" +akka.actor.deployment.service-hello.clustered.preferred-nodes = ["node:node2"] +akka.actor.deployment.service-hello.clustered.replication-factor = 2 \ No newline at end of file diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_3_replicas/RoundRobin3ReplicasMultiJvmNode3.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmNode3.opts similarity index 100% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_3_replicas/RoundRobin3ReplicasMultiJvmNode3.opts rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmNode3.opts diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmSpec.scala new file mode 100644 index 0000000000..27dc35a8f6 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmSpec.scala @@ -0,0 +1,119 @@ +package akka.cluster.routing.random.failover + +import akka.config.Config +import akka.cluster._ +import akka.actor.{ ActorRef, Actor } +import java.util.{ Collections, Set ⇒ JSet } + +object RandomFailoverMultiJvmSpec { + + val NrOfNodes = 3 + + class SomeActor extends Actor with Serializable { + println("---------------------------------------------------------------------------") + println("SomeActor has been created on node [" + Config.nodename + "]") + println("---------------------------------------------------------------------------") + + def receive = { + case "identify" ⇒ { + println("The node received the 'identify' command") + self.reply(Config.nodename) + } + case "shutdown" ⇒ { + println("The node received the 'shutdown' command") + Cluster.node.shutdown() + } + } + } + +} + +class RandomFailoverMultiJvmNode1 extends MasterClusterTestNode { + + import RandomFailoverMultiJvmSpec._ + + def testNodes = NrOfNodes + + def sleepSome() { + println("Starting sleep") + Thread.sleep(1000) //nasty.. but ok for now. + println("Finished doing sleep") + } + + "Random: when routing fails" must { + "jump to another replica" in { + Cluster.node.start() + Cluster.barrier("waiting-for-begin", NrOfNodes).await() + + // ============= the real testing ================= + val actor = Actor.actorOf[SomeActor]("service-hello").asInstanceOf[ClusterActorRef] + + val oldFoundConnections = identifyConnections(actor) + println("---------------------------- oldFoundConnections ------------------------") + println(oldFoundConnections) + + //since we have replication factor 2 + oldFoundConnections.size() must be(2) + + //terminate a node + actor ! "shutdown" + + sleepSome() + + //this is where the system behaves unpredictable. From time to time it works... from time to time there + //all kinds of connection timeouts. So this test shows that there are problems. For the time being + //the test code has been deactivated to prevent causing problems. + + val newFoundConnections = identifyConnections(actor) + println("---------------------------- newFoundConnections ------------------------") + println(newFoundConnections) + + //it still must be 2 since a different node should have been used to failover to + newFoundConnections.size() must be(2) + //they are not disjoint since, there must be a single element that is in both + Collections.disjoint(newFoundConnections, oldFoundConnections) must be(false) + //but they should not be equal since the shutdown-node has been replaced by another one. + newFoundConnections.equals(oldFoundConnections) must be(false) + + Cluster.node.shutdown() + } + } + + def identifyConnections(actor: ActorRef): JSet[String] = { + val set = new java.util.HashSet[String] + for (i ← 0 until NrOfNodes * 10) { + val value = (actor ? "identify").get.asInstanceOf[String] + set.add(value) + } + set + } +} + +class RandomFailoverMultiJvmNode2 extends ClusterTestNode { + + import RandomFailoverMultiJvmSpec._ + + "___" must { + "___" in { + Cluster.node.start() + Cluster.barrier("waiting-for-begin", NrOfNodes).await() + + Thread.sleep(30 * 1000) + } + } +} + +class RandomFailoverMultiJvmNode3 extends ClusterTestNode { + + import RandomFailoverMultiJvmSpec._ + + "___" must { + "___" in { + Cluster.node.start() + Cluster.barrier("waiting-for-begin", NrOfNodes).await() + + Thread.sleep(30 * 1000) + } + } +} + diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/homenode/HomeNodeMultiJvmNode1.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/homenode/HomeNodeMultiJvmNode1.conf new file mode 100644 index 0000000000..74ca26985f --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/homenode/HomeNodeMultiJvmNode1.conf @@ -0,0 +1,8 @@ +akka.enabled-modules = ["cluster"] +akka.event-handler-level = "WARNING" +akka.actor.deployment.service-node1.router = "random" +akka.actor.deployment.service-node1.clustered.preferred-nodes = ["node:node1"] +akka.actor.deployment.service-node1.clustered.replication-factor = 1 +akka.actor.deployment.service-node2.router = "random" +akka.actor.deployment.service-node2.clustered.preferred-nodes = ["node:node2"] +akka.actor.deployment.service-node2.clustered.replication-factor = 1 \ No newline at end of file diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_1_replica/RoundRobin1ReplicaMultiJvmNode1.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/homenode/HomeNodeMultiJvmNode1.opts similarity index 100% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_1_replica/RoundRobin1ReplicaMultiJvmNode1.opts rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/homenode/HomeNodeMultiJvmNode1.opts diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/homenode/HomeNodeMultiJvmNode2.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/homenode/HomeNodeMultiJvmNode2.conf new file mode 100644 index 0000000000..74ca26985f --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/homenode/HomeNodeMultiJvmNode2.conf @@ -0,0 +1,8 @@ +akka.enabled-modules = ["cluster"] +akka.event-handler-level = "WARNING" +akka.actor.deployment.service-node1.router = "random" +akka.actor.deployment.service-node1.clustered.preferred-nodes = ["node:node1"] +akka.actor.deployment.service-node1.clustered.replication-factor = 1 +akka.actor.deployment.service-node2.router = "random" +akka.actor.deployment.service-node2.clustered.preferred-nodes = ["node:node2"] +akka.actor.deployment.service-node2.clustered.replication-factor = 1 \ No newline at end of file diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_3_replicas/RoundRobin3ReplicasMultiJvmNode2.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/homenode/HomeNodeMultiJvmNode2.opts similarity index 100% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_3_replicas/RoundRobin3ReplicasMultiJvmNode2.opts rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/homenode/HomeNodeMultiJvmNode2.opts diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/homenode/HomeNodeMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/homenode/HomeNodeMultiJvmSpec.scala new file mode 100644 index 0000000000..504eb594d3 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/homenode/HomeNodeMultiJvmSpec.scala @@ -0,0 +1,59 @@ +package akka.cluster.routing.random.homenode + +import akka.config.Config +import akka.actor.Actor +import akka.cluster.{ ClusterTestNode, MasterClusterTestNode, Cluster } +import Cluster._ + +object HomeNodeMultiJvmSpec { + + val NrOfNodes = 2 + + class SomeActor extends Actor with Serializable { + def receive = { + case "identify" ⇒ { + self.reply(Config.nodename) + } + } + } + +} + +class HomeNodeMultiJvmNode1 extends MasterClusterTestNode { + + import HomeNodeMultiJvmSpec._ + + val testNodes = NrOfNodes + + "___" must { + "___" in { + node.start() + barrier("waiting-for-begin", NrOfNodes).await() + barrier("waiting-for-end", NrOfNodes).await() + node.shutdown() + } + } +} + +class HomeNodeMultiJvmNode2 extends ClusterTestNode { + + import HomeNodeMultiJvmSpec._ + + "Random Router: A Random Router" must { + "obey 'home-node' config option when instantiated actor in cluster" in { + node.start() + barrier("waiting-for-begin", NrOfNodes).await() + + val actorNode1 = Actor.actorOf[SomeActor]("service-node1") + val nameNode1 = (actorNode1 ? "identify").get.asInstanceOf[String] + nameNode1 must equal("node1") + + val actorNode2 = Actor.actorOf[SomeActor]("service-node2") + val nameNode2 = (actorNode2 ? "identify").get.asInstanceOf[String] + nameNode2 must equal("node2") + + barrier("waiting-for-end", NrOfNodes).await() + node.shutdown() + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/single_replica/SingleReplicaDirectRoutingMultiJvmNode1.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_1/Random1ReplicaMultiJvmNode1.conf similarity index 72% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/single_replica/SingleReplicaDirectRoutingMultiJvmNode1.conf rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_1/Random1ReplicaMultiJvmNode1.conf index 0a47105d03..4140cf7450 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/single_replica/SingleReplicaDirectRoutingMultiJvmNode1.conf +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_1/Random1ReplicaMultiJvmNode1.conf @@ -1,4 +1,4 @@ akka.enabled-modules = ["cluster"] akka.event-handler-level = "WARNING" -akka.actor.deployment.service-hello.router = "direct" +akka.actor.deployment.service-hello.router = "random" akka.actor.deployment.service-hello.clustered.replication-factor = 1 \ No newline at end of file diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_2_replicas/RoundRobin2ReplicasMultiJvmNode1.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_1/Random1ReplicaMultiJvmNode1.opts similarity index 100% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_2_replicas/RoundRobin2ReplicasMultiJvmNode1.opts rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_1/Random1ReplicaMultiJvmNode1.opts diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_1/Random1ReplicaMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_1/Random1ReplicaMultiJvmSpec.scala new file mode 100644 index 0000000000..ce5a603555 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_1/Random1ReplicaMultiJvmSpec.scala @@ -0,0 +1,50 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ + +package akka.cluster.routing.random.replicationfactor_1 + +import akka.cluster._ +import akka.cluster.Cluster._ +import akka.actor._ +import akka.config.Config + +/** + * Test that if a single node is used with a random router with replication factor then the actor is instantiated + * on the single node. + */ +object Random1ReplicaMultiJvmSpec { + + class HelloWorld extends Actor with Serializable { + def receive = { + case "Hello" ⇒ + self.reply("World from node [" + Config.nodename + "]") + } + } + +} + +class Random1ReplicaMultiJvmNode1 extends MasterClusterTestNode { + + import Random1ReplicaMultiJvmSpec._ + + val testNodes = 1 + + "Random Router: A cluster" must { + + "create clustered actor, get a 'local' actor on 'home' node and a 'ref' to actor on remote node" in { + node.start() + + var hello = Actor.actorOf[HelloWorld]("service-hello") + hello must not equal (null) + hello.address must equal("service-hello") + hello.isInstanceOf[ClusterActorRef] must be(true) + + hello must not equal (null) + val reply = (hello ? "Hello").as[String].getOrElse(fail("Should have recieved reply from node1")) + reply must equal("World from node [node1]") + + node.shutdown() + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/bad_address/BadAddressDirectRoutingMultiJvmNode1.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmNode1.conf similarity index 51% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/bad_address/BadAddressDirectRoutingMultiJvmNode1.conf rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmNode1.conf index e12f87e912..3efc8089a1 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/bad_address/BadAddressDirectRoutingMultiJvmNode1.conf +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmNode1.conf @@ -1,5 +1,4 @@ akka.enabled-modules = ["cluster"] akka.event-handler-level = "WARNING" -akka.actor.deployment.service-hello.router = "round-robin" -akka.actor.deployment.service-hello.clustered.home = "node:node1" -akka.actor.deployment.service-hello.clustered.replication-factor = 1 \ No newline at end of file +akka.actor.deployment.service-hello.router = "random" +akka.actor.deployment.service-hello.clustered.replication-factor = 3 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_3_replicas/RoundRobin3ReplicasMultiJvmNode1.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmNode1.opts similarity index 100% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_3_replicas/RoundRobin3ReplicasMultiJvmNode1.opts rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmNode1.opts diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmNode2.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmNode2.conf new file mode 100644 index 0000000000..365bc678d8 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmNode2.conf @@ -0,0 +1,4 @@ +akka.enabled-modules = ["cluster"] +akka.event-handler-level = "WARNING" +akka.actor.deployment.service-hello.router = "random" +akka.actor.deployment.service-hello.clustered.repliction-factor = 3 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmNode2.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmNode2.opts similarity index 100% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmNode2.opts rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmNode2.opts diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmNode3.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmNode3.conf new file mode 100644 index 0000000000..3efc8089a1 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmNode3.conf @@ -0,0 +1,4 @@ +akka.enabled-modules = ["cluster"] +akka.event-handler-level = "WARNING" +akka.actor.deployment.service-hello.router = "random" +akka.actor.deployment.service-hello.clustered.replication-factor = 3 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmNode3.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmNode3.opts similarity index 100% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmNode3.opts rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmNode3.opts diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmSpec.scala new file mode 100644 index 0000000000..9bc162ab52 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmSpec.scala @@ -0,0 +1,111 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ + +package akka.cluster.routing.random.replicationfactor_3 + +import akka.cluster._ +import akka.actor._ +import akka.config.Config +import Cluster._ + +/** + * When a MultiJvmNode is started, will it automatically be part of the cluster (so will it automatically be eligible + * for running actors, or will it be just a 'client' talking to the cluster. + */ +object Random3ReplicasMultiJvmSpec { + val NrOfNodes = 3 + + class HelloWorld extends Actor with Serializable { + def receive = { + case "Hello" ⇒ + self.reply("World from node [" + Config.nodename + "]") + } + } +} + +/** + * What is the purpose of this node? Is this just a node for the cluster to make use of? + */ +class Random3ReplicasMultiJvmNode1 extends MasterClusterTestNode { + + import Random3ReplicasMultiJvmSpec._ + + def testNodes: Int = NrOfNodes + + "___" must { + "___" in { + node.start() + + //wait till node 1 has started. + barrier("begin", NrOfNodes).await() + + barrier("end", NrOfNodes).await() + + node.shutdown() + } + } +} + +class Random3ReplicasMultiJvmNode2 extends ClusterTestNode { + + import Random3ReplicasMultiJvmSpec._ + import Cluster._ + + "Random: A cluster" must { + + "distribute requests randomly" in { + node.start() + + //wait till node 1 has started. + barrier("begin", NrOfNodes).await() + + //check if the actorRef is the expected remoteActorRef. + var hello: ActorRef = null + hello = Actor.actorOf[HelloWorld]("service-hello") + hello must not equal (null) + hello.address must equal("service-hello") + hello.isInstanceOf[ClusterActorRef] must be(true) + + //todo: is there a reason to check for null again since it already has been done in the previous block. + hello must not equal (null) + + val replies = collection.mutable.Map.empty[String, Int] + def count(reply: String) = { + if (replies.get(reply).isEmpty) replies.put(reply, 1) + else replies.put(reply, replies(reply) + 1) + } + + for (i ← 0 until 1000) { + count((hello ? "Hello").as[String].getOrElse(fail("Should have recieved reply from a node"))) + } + + assert(replies("World from node [node1]") > 100) + assert(replies("World from node [node2]") > 100) + assert(replies("World from node [node3]") > 100) + + barrier("end", NrOfNodes).await() + + node.shutdown() + } + } +} + +class Random3ReplicasMultiJvmNode3 extends ClusterTestNode { + + import Random3ReplicasMultiJvmSpec._ + import Cluster._ + + "___" must { + "___" in { + node.start() + + //wait till node 1 has started. + barrier("begin", NrOfNodes).await() + + barrier("end", NrOfNodes).await() + + node.shutdown() + } + } +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmNode1.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmNode1.conf similarity index 93% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmNode1.conf rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmNode1.conf index 366cf9111c..f70ecc2a65 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmNode1.conf +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmNode1.conf @@ -1,5 +1,5 @@ akka.enabled-modules = ["cluster"] akka.event-handler-level = "WARNING" akka.actor.deployment.service-hello.router = "round-robin" -akka.actor.deployment.service-hello.clustered.preferred-nodes = ["node:node1"] -akka.actor.deployment.service-hello.clustered.replication-factor = 1 \ No newline at end of file +akka.actor.deployment.service-hello.clustered.preferred-nodes = ["node:node2"] +akka.actor.deployment.service-hello.clustered.replication-factor = 2 \ No newline at end of file diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmNode1.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmNode1.opts similarity index 100% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmNode1.opts rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmNode1.opts diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmNode2.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmNode2.conf similarity index 93% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmNode2.conf rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmNode2.conf index 366cf9111c..f70ecc2a65 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmNode2.conf +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmNode2.conf @@ -1,5 +1,5 @@ akka.enabled-modules = ["cluster"] akka.event-handler-level = "WARNING" akka.actor.deployment.service-hello.router = "round-robin" -akka.actor.deployment.service-hello.clustered.preferred-nodes = ["node:node1"] -akka.actor.deployment.service-hello.clustered.replication-factor = 1 \ No newline at end of file +akka.actor.deployment.service-hello.clustered.preferred-nodes = ["node:node2"] +akka.actor.deployment.service-hello.clustered.replication-factor = 2 \ No newline at end of file diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmNode2.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmNode2.opts new file mode 100644 index 0000000000..f1e01f253d --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmNode2.opts @@ -0,0 +1 @@ +-Dakka.cluster.nodename=node2 -Dakka.cluster.port=9992 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmNode3.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmNode3.conf similarity index 93% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmNode3.conf rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmNode3.conf index 366cf9111c..f70ecc2a65 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmNode3.conf +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmNode3.conf @@ -1,5 +1,5 @@ akka.enabled-modules = ["cluster"] akka.event-handler-level = "WARNING" akka.actor.deployment.service-hello.router = "round-robin" -akka.actor.deployment.service-hello.clustered.preferred-nodes = ["node:node1"] -akka.actor.deployment.service-hello.clustered.replication-factor = 1 \ No newline at end of file +akka.actor.deployment.service-hello.clustered.preferred-nodes = ["node:node2"] +akka.actor.deployment.service-hello.clustered.replication-factor = 2 \ No newline at end of file diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmNode3.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmNode3.opts new file mode 100644 index 0000000000..202496ad31 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmNode3.opts @@ -0,0 +1 @@ +-Dakka.cluster.nodename=node3 -Dakka.cluster.port=9993 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmSpec.scala new file mode 100644 index 0000000000..f8c5fc0380 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmSpec.scala @@ -0,0 +1,120 @@ +package akka.cluster.routing.roundrobin.failover + +import akka.config.Config +import akka.cluster._ +import akka.actor.{ ActorRef, Actor } +import java.util.{ Collections, Set ⇒ JSet } + +object RoundRobinFailoverMultiJvmSpec { + + val NrOfNodes = 3 + + class SomeActor extends Actor with Serializable { + println("---------------------------------------------------------------------------") + println("SomeActor has been created on node [" + Config.nodename + "]") + println("---------------------------------------------------------------------------") + + def receive = { + case "identify" ⇒ { + println("The node received the 'identify' command") + self.reply(Config.nodename) + } + case "shutdown" ⇒ { + println("The node received the 'shutdown' command") + Cluster.node.shutdown() + } + } + } + +} + +class RoundRobinFailoverMultiJvmNode1 extends MasterClusterTestNode { + + import RoundRobinFailoverMultiJvmSpec._ + + def testNodes = NrOfNodes + + def sleepSome() { + println("Starting sleep") + Thread.sleep(1000) //nasty.. but ok for now. + println("Finished doing sleep") + } + + "Round Robin: when round robin fails" must { + "jump to another replica" in { + Cluster.node.start() + Cluster.barrier("waiting-for-begin", NrOfNodes).await() + + // ============= the real testing ================= + val actor = Actor.actorOf[SomeActor]("service-hello").asInstanceOf[ClusterActorRef] + + val oldFoundConnections = identifyConnections(actor) + println("---------------------------- oldFoundConnections ------------------------") + println(oldFoundConnections) + + //since we have replication factor 2 + oldFoundConnections.size() must be(2) + + //terminate a node + actor ! "shutdown" + + sleepSome() + + //this is where the system behaves unpredictable. From time to time it works... from time to time there + //all kinds of connection timeouts. So this test shows that there are problems. For the time being + //the test code has been deactivated to prevent causing problems. + + /* + val newFoundConnections = identifyConnections(actor) + println("---------------------------- newFoundConnections ------------------------") + println(newFoundConnections) + + //it still must be 2 since a different node should have been used to failover to + newFoundConnections.size() must be(2) + //they are not disjoint since, there must be a single element that is in both + Collections.disjoint(newFoundConnections, oldFoundConnections) must be(false) + //but they should not be equal since the shutdown-node has been replaced by another one. + newFoundConnections.equals(oldFoundConnections) must be(false) + */ + Cluster.node.shutdown() + } + } + + def identifyConnections(actor: ActorRef): JSet[String] = { + val set = new java.util.HashSet[String] + for (i ← 0 until NrOfNodes * 2) { + val value = (actor ? "identify").get.asInstanceOf[String] + set.add(value) + } + set + } +} + +class RoundRobinFailoverMultiJvmNode2 extends ClusterTestNode { + + import RoundRobinFailoverMultiJvmSpec._ + + "___" must { + "___" in { + Cluster.node.start() + Cluster.barrier("waiting-for-begin", NrOfNodes).await() + + Thread.sleep(30 * 1000) + } + } +} + +class RoundRobinFailoverMultiJvmNode3 extends ClusterTestNode { + + import RoundRobinFailoverMultiJvmSpec._ + + "___" must { + "___" in { + Cluster.node.start() + Cluster.barrier("waiting-for-begin", NrOfNodes).await() + + Thread.sleep(30 * 1000) + } + } +} + diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/homenode/HomeNodeMultiJvmNode1.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/homenode/HomeNodeMultiJvmNode1.conf new file mode 100644 index 0000000000..c1028656d3 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/homenode/HomeNodeMultiJvmNode1.conf @@ -0,0 +1,8 @@ +akka.enabled-modules = ["cluster"] +akka.event-handler-level = "WARNING" +akka.actor.deployment.service-node1.router = "round-robin" +akka.actor.deployment.service-node1.clustered.preferred-nodes = ["node:node1"] +akka.actor.deployment.service-node1.clustered.replication-factor = 1 +akka.actor.deployment.service-node2.router = "round-robin" +akka.actor.deployment.service-node2.clustered.preferred-nodes = ["node:node2"] +akka.actor.deployment.service-node2.clustered.replication-factor = 1 \ No newline at end of file diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/homenode/HomeNodeMultiJvmNode1.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/homenode/HomeNodeMultiJvmNode1.opts new file mode 100644 index 0000000000..a88c260d8c --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/homenode/HomeNodeMultiJvmNode1.opts @@ -0,0 +1 @@ +-Dakka.cluster.nodename=node1 -Dakka.cluster.port=9991 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/homenode/HomeNodeMultiJvmNode2.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/homenode/HomeNodeMultiJvmNode2.conf similarity index 100% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/homenode/HomeNodeMultiJvmNode2.conf rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/homenode/HomeNodeMultiJvmNode2.conf diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/homenode/HomeNodeMultiJvmNode2.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/homenode/HomeNodeMultiJvmNode2.opts new file mode 100644 index 0000000000..f1e01f253d --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/homenode/HomeNodeMultiJvmNode2.opts @@ -0,0 +1 @@ +-Dakka.cluster.nodename=node2 -Dakka.cluster.port=9992 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/homenode/HomeNodeMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/homenode/HomeNodeMultiJvmSpec.scala similarity index 59% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/homenode/HomeNodeMultiJvmSpec.scala rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/homenode/HomeNodeMultiJvmSpec.scala index e4aae69f8f..45acd08279 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/homenode/HomeNodeMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/homenode/HomeNodeMultiJvmSpec.scala @@ -1,7 +1,7 @@ -package akka.cluster.routing.homenode +package akka.cluster.routing.roundrobin.homenode import akka.config.Config -import akka.actor.{ ActorRef, Actor } +import akka.actor.Actor import akka.cluster.{ ClusterTestNode, MasterClusterTestNode, Cluster } import Cluster._ @@ -16,6 +16,7 @@ object HomeNodeMultiJvmSpec { } } } + } class HomeNodeMultiJvmNode1 extends MasterClusterTestNode { @@ -24,13 +25,12 @@ class HomeNodeMultiJvmNode1 extends MasterClusterTestNode { val testNodes = NrOfNodes - "A Router" must { - "obey 'home-node' config option when instantiated actor in cluster" in { + "___" must { + "___" in { node.start() barrier("waiting-for-begin", NrOfNodes).await() - - barrier("get-ref-to-actor-on-node2", NrOfNodes).await() + barrier("waiting-for-end", NrOfNodes).await() node.shutdown() } @@ -41,18 +41,21 @@ class HomeNodeMultiJvmNode2 extends ClusterTestNode { import HomeNodeMultiJvmSpec._ - "A Router" must { + "Round Robin: A Router" must { "obey 'home-node' config option when instantiated actor in cluster" in { node.start() barrier("waiting-for-begin", NrOfNodes).await() - barrier("get-ref-to-actor-on-node2", NrOfNodes) { - val actor = Actor.actorOf[SomeActor]("service-hello") - val name = (actor ? "identify").get.asInstanceOf[String] - name must equal("node1") - } + val actorNode1 = Actor.actorOf[SomeActor]("service-node1") + val name1 = (actorNode1 ? "identify").get.asInstanceOf[String] + name1 must equal("node1") + val actorNode2 = Actor.actorOf[SomeActor]("service-node2") + val name2 = (actorNode2 ? "identify").get.asInstanceOf[String] + name2 must equal("node2") + + barrier("waiting-for-end", NrOfNodes).await() node.shutdown() } } diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_1_replica/RoundRobin1ReplicaMultiJvmNode1.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_1/RoundRobin1ReplicaMultiJvmNode1.conf similarity index 100% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_1_replica/RoundRobin1ReplicaMultiJvmNode1.conf rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_1/RoundRobin1ReplicaMultiJvmNode1.conf diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_1/RoundRobin1ReplicaMultiJvmNode1.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_1/RoundRobin1ReplicaMultiJvmNode1.opts new file mode 100644 index 0000000000..a88c260d8c --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_1/RoundRobin1ReplicaMultiJvmNode1.opts @@ -0,0 +1 @@ +-Dakka.cluster.nodename=node1 -Dakka.cluster.port=9991 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_1_replica/RoundRobin1ReplicaMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_1/RoundRobin1ReplicaMultiJvmSpec.scala similarity index 88% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_1_replica/RoundRobin1ReplicaMultiJvmSpec.scala rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_1/RoundRobin1ReplicaMultiJvmSpec.scala index 9a9c1dfbb2..bc33d0c078 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_1_replica/RoundRobin1ReplicaMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_1/RoundRobin1ReplicaMultiJvmSpec.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2011 Typesafe Inc. */ -package akka.cluster.routing.roundrobin_1_replica +package akka.cluster.routing.roundrobin.replicationfactor_1 import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers @@ -32,13 +32,12 @@ class RoundRobin1ReplicaMultiJvmNode1 extends MasterClusterTestNode { val testNodes = 1 - "A cluster" must { + "Round Robin: A cluster" must { "create clustered actor, get a 'local' actor on 'home' node and a 'ref' to actor on remote node" in { node.start() - var hello: ActorRef = null - hello = Actor.actorOf[HelloWorld]("service-hello") + var hello = Actor.actorOf[HelloWorld]("service-hello") hello must not equal (null) hello.address must equal("service-hello") hello.isInstanceOf[ClusterActorRef] must be(true) diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_2_replicas/RoundRobin2ReplicasMultiJvmNode1.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_2/RoundRobin2ReplicasMultiJvmNode1.conf similarity index 100% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_2_replicas/RoundRobin2ReplicasMultiJvmNode1.conf rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_2/RoundRobin2ReplicasMultiJvmNode1.conf diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_2/RoundRobin2ReplicasMultiJvmNode1.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_2/RoundRobin2ReplicasMultiJvmNode1.opts new file mode 100644 index 0000000000..a88c260d8c --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_2/RoundRobin2ReplicasMultiJvmNode1.opts @@ -0,0 +1 @@ +-Dakka.cluster.nodename=node1 -Dakka.cluster.port=9991 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_2_replicas/RoundRobin2ReplicasMultiJvmNode2.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_2/RoundRobin2ReplicasMultiJvmNode2.conf similarity index 100% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_2_replicas/RoundRobin2ReplicasMultiJvmNode2.conf rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_2/RoundRobin2ReplicasMultiJvmNode2.conf diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_2/RoundRobin2ReplicasMultiJvmNode2.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_2/RoundRobin2ReplicasMultiJvmNode2.opts new file mode 100644 index 0000000000..f1e01f253d --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_2/RoundRobin2ReplicasMultiJvmNode2.opts @@ -0,0 +1 @@ +-Dakka.cluster.nodename=node2 -Dakka.cluster.port=9992 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_2_replicas/RoundRobin2ReplicasMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_2/RoundRobin2ReplicasMultiJvmSpec.scala similarity index 90% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_2_replicas/RoundRobin2ReplicasMultiJvmSpec.scala rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_2/RoundRobin2ReplicasMultiJvmSpec.scala index aac06caa90..213ad85382 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_2_replicas/RoundRobin2ReplicasMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_2/RoundRobin2ReplicasMultiJvmSpec.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2011 Typesafe Inc. */ -package akka.cluster.routing.roundrobin_2_replicas +package akka.cluster.routing.roundrobin.replicationfactor_2 import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers @@ -39,7 +39,7 @@ object RoundRobin2ReplicasMultiJvmSpec { class RoundRobin2ReplicasMultiJvmNode1 extends WordSpec with MustMatchers with BeforeAndAfterAll { import RoundRobin2ReplicasMultiJvmSpec._ - "A cluster" must { + "Round Robin: A cluster" must { "create clustered actor, get a 'local' actor on 'home' node and a 'ref' to actor on remote node" in { System.getProperty("akka.cluster.nodename", "") must be("node1") @@ -51,16 +51,16 @@ class RoundRobin2ReplicasMultiJvmNode1 extends WordSpec with MustMatchers with B } //wait till ndoe 2 has started. - barrier("start-node2", NrOfNodes) {} + barrier("start-node2", NrOfNodes).await() //wait till node 3 has started. - barrier("start-node3", NrOfNodes) {} + barrier("start-node3", NrOfNodes).await() //wait till an actor reference on node 2 has become available. - barrier("get-ref-to-actor-on-node2", NrOfNodes) {} + barrier("get-ref-to-actor-on-node2", NrOfNodes).await() //wait till the node 2 has send a message to the replica's. - barrier("send-message-from-node2-to-replicas", NrOfNodes) {} + barrier("send-message-from-node2-to-replicas", NrOfNodes).await() node.shutdown() } @@ -78,14 +78,14 @@ class RoundRobin2ReplicasMultiJvmNode1 extends WordSpec with MustMatchers with B class RoundRobin2ReplicasMultiJvmNode2 extends WordSpec with MustMatchers { import RoundRobin2ReplicasMultiJvmSpec._ - "A cluster" must { + "Round Robin: A cluster" must { "create clustered actor, get a 'local' actor on 'home' node and a 'ref' to actor on remote node" in { System.getProperty("akka.cluster.nodename", "") must be("node2") System.getProperty("akka.cluster.port", "") must be("9992") //wait till node 1 has started. - barrier("start-node1", NrOfNodes) {} + barrier("start-node1", NrOfNodes).await() //wait till node 2 has started. barrier("start-node2", NrOfNodes) { @@ -93,7 +93,7 @@ class RoundRobin2ReplicasMultiJvmNode2 extends WordSpec with MustMatchers { } //wait till node 3 has started. - barrier("start-node3", NrOfNodes) {} + barrier("start-node3", NrOfNodes).await() //check if the actorRef is the expected remoteActorRef. var hello: ActorRef = null diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_3_replicas/RoundRobin3ReplicasMultiJvmNode1.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmNode1.conf similarity index 100% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_3_replicas/RoundRobin3ReplicasMultiJvmNode1.conf rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmNode1.conf diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmNode1.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmNode1.opts new file mode 100644 index 0000000000..a88c260d8c --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmNode1.opts @@ -0,0 +1 @@ +-Dakka.cluster.nodename=node1 -Dakka.cluster.port=9991 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_3_replicas/RoundRobin3ReplicasMultiJvmNode2.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmNode2.conf similarity index 100% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_3_replicas/RoundRobin3ReplicasMultiJvmNode2.conf rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmNode2.conf diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmNode2.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmNode2.opts new file mode 100644 index 0000000000..f1e01f253d --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmNode2.opts @@ -0,0 +1 @@ +-Dakka.cluster.nodename=node2 -Dakka.cluster.port=9992 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_3_replicas/RoundRobin3ReplicasMultiJvmNode3.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmNode3.conf similarity index 100% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_3_replicas/RoundRobin3ReplicasMultiJvmNode3.conf rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmNode3.conf diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmNode3.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmNode3.opts new file mode 100644 index 0000000000..202496ad31 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmNode3.opts @@ -0,0 +1 @@ +-Dakka.cluster.nodename=node3 -Dakka.cluster.port=9993 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_3_replicas/RoundRobin3ReplicasMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmSpec.scala similarity index 86% rename from akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_3_replicas/RoundRobin3ReplicasMultiJvmSpec.scala rename to akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmSpec.scala index d3d5a48cbf..88a72c0735 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_3_replicas/RoundRobin3ReplicasMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_3/RoundRobin3ReplicasMultiJvmSpec.scala @@ -2,7 +2,7 @@ * Copyright (C) 2009-2011 Typesafe Inc. */ -package akka.cluster.routing.roundrobin_3_replicas +package akka.cluster.routing.roundrobin.replicationfactor_3 import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers @@ -35,7 +35,7 @@ object RoundRobin3ReplicasMultiJvmSpec { class RoundRobin3ReplicasMultiJvmNode1 extends WordSpec with MustMatchers with BeforeAndAfterAll { import RoundRobin3ReplicasMultiJvmSpec._ - "A cluster" must { + "Round Robin: A cluster" must { "create clustered actor, get a 'local' actor on 'home' node and a 'ref' to actor on remote node" ignore { @@ -45,16 +45,16 @@ class RoundRobin3ReplicasMultiJvmNode1 extends WordSpec with MustMatchers with B } //wait till ndoe 2 has started. - barrier("start-node2", NrOfNodes) {} + barrier("start-node2", NrOfNodes).await() //wait till node 3 has started. - barrier("start-node3", NrOfNodes) {} + barrier("start-node3", NrOfNodes).await() //wait till an actor reference on node 2 has become available. - barrier("get-ref-to-actor-on-node2", NrOfNodes) {} + barrier("get-ref-to-actor-on-node2", NrOfNodes).await() //wait till the node 2 has send a message to the replica's. - barrier("send-message-from-node2-to-replicas", NrOfNodes) {} + barrier("send-message-from-node2-to-replicas", NrOfNodes).await() node.shutdown() } @@ -73,12 +73,12 @@ class RoundRobin3ReplicasMultiJvmNode2 extends WordSpec with MustMatchers { import RoundRobin3ReplicasMultiJvmSpec._ import Cluster._ - "A cluster" must { + "Round Robin: A cluster" must { "create clustered actor, get a 'local' actor on 'home' node and a 'ref' to actor on remote node" ignore { //wait till node 1 has started. - barrier("start-node1", NrOfNodes) {} + barrier("start-node1", NrOfNodes).await() //wait till node 2 has started. barrier("start-node2", NrOfNodes) { @@ -86,7 +86,7 @@ class RoundRobin3ReplicasMultiJvmNode2 extends WordSpec with MustMatchers { } //wait till node 3 has started. - barrier("start-node3", NrOfNodes) {} + barrier("start-node3", NrOfNodes).await() //check if the actorRef is the expected remoteActorRef. var hello: ActorRef = null @@ -134,20 +134,20 @@ class RoundRobin3ReplicasMultiJvmNode3 extends WordSpec with MustMatchers { import RoundRobin3ReplicasMultiJvmSpec._ import Cluster._ - "A cluster" must { + "Round Robin: A cluster" must { "create clustered actor, get a 'local' actor on 'home' node and a 'ref' to actor on remote node" ignore { - barrier("start-node1", NrOfNodes) {} + barrier("start-node1", NrOfNodes).await() - barrier("start-node2", NrOfNodes) {} + barrier("start-node2", NrOfNodes).await() barrier("start-node3", NrOfNodes) { node.start() } - barrier("get-ref-to-actor-on-node2", NrOfNodes) {} + barrier("get-ref-to-actor-on-node2", NrOfNodes).await() - barrier("send-message-from-node2-to-replicas", NrOfNodes) {} + barrier("send-message-from-node2-to-replicas", NrOfNodes).await() node.shutdown() } diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmNode4.opts b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmNode4.opts deleted file mode 100644 index 8c875faf53..0000000000 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmNode4.opts +++ /dev/null @@ -1 +0,0 @@ --Dakka.cluster.nodename=node4 -Dakka.cluster.port=9994 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmSpec.scala deleted file mode 100644 index 30a1903096..0000000000 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/RoundRobinFailoverMultiJvmSpec.scala +++ /dev/null @@ -1,144 +0,0 @@ -package akka.cluster.routing.roundrobin_failover - -import akka.config.Config -import akka.cluster._ -import akka.actor.{ ActorRef, Actor } - -object RoundRobinFailoverMultiJvmSpec { - - val NrOfNodes = 2 - - class SomeActor extends Actor with Serializable { - println("---------------------------------------------------------------------------") - println("SomeActor has been created on node [" + Config.nodename + "]") - println("---------------------------------------------------------------------------") - - def receive = { - case "identify" ⇒ { - println("The node received the 'identify' command") - self.reply(Config.nodename) - } - case "shutdown" ⇒ { - println("The node received the 'shutdown' command") - Cluster.node.shutdown() - } - } - } -} - -class RoundRobinFailoverMultiJvmNode1 extends MasterClusterTestNode { - - import RoundRobinFailoverMultiJvmSpec._ - - val testNodes = NrOfNodes - - "foo" must { - "bla" in { - Cluster.node.start() - Cluster.barrier("waiting-for-begin", NrOfNodes).await() - - println("Getting reference to service-hello actor") - var hello: ActorRef = null - Cluster.barrier("get-ref-to-actor-on-node2", NrOfNodes) { - hello = Actor.actorOf[SomeActor]("service-hello") - } - - println("Successfully acquired reference") - - Cluster.barrier("waiting-to-end", NrOfNodes).await() - Cluster.node.shutdown() - } - } -} - -class RoundRobinFailoverMultiJvmNode2 extends ClusterTestNode { - - import RoundRobinFailoverMultiJvmSpec._ - - "foo" must { - "bla" in { - println("Started Zookeeper Node") - Cluster.node.start() - println("Waiting to begin") - Cluster.barrier("waiting-for-begin", NrOfNodes).await() - println("Begin!") - - Cluster.barrier("get-ref-to-actor-on-node2", NrOfNodes).await() - - // ============= the real testing ================= - /* - val actor = Actor.actorOf[SomeActor]("service-hello") - val firstTimeResult = (actor ? "identify").get - val secondTimeResult = (actor ? "identify").get - //since there are only 2 nodes, the identity should not have changed. - assert(firstTimeResult == secondTimeResult) - - //if we now terminate the node that - actor ! "shutdown" - - //todo: do some waiting - println("Doing some sleep") - try { - Thread.sleep(4000) //nasty.. but ok for now. - println("Finished doing sleep") - } finally { - println("Ended the Thread.sleep method somehow..") - } - - //now we should get a different node that responds to us since there was a failover. - val thirdTimeResult = (actor ? "identify").get - assert(!(firstTimeResult == thirdTimeResult)) */ - // ================================================== - - println("Waiting to end") - Cluster.barrier("waiting-to-end", NrOfNodes).await() - println("Shutting down ClusterNode") - Cluster.node.shutdown() - } - } -} - -/* -class RoundRobinFailoverMultiJvmNode3 extends SlaveNode { - - import RoundRobinFailoverMultiJvmSpec._ - - "foo" must { - "bla" in { - println("Started Zookeeper Node") - Cluster.node.start() - println("Waiting to begin") - Cluster.barrier("waiting-for-begin", NrOfNodes).await() - println("Begin!") - - Cluster.barrier("get-ref-to-actor-on-node2", NrOfNodes).await() - - println("Waiting to end") - Cluster.barrier("waiting-to-end", NrOfNodes).await() - println("Shutting down ClusterNode") - Cluster.node.shutdown() - } - } -} - -class RoundRobinFailoverMultiJvmNode4 extends SlaveNode { - - import RoundRobinFailoverMultiJvmSpec._ - - "foo" must { - "bla" in { - println("Started Zookeeper Node") - Cluster.node.start() - println("Waiting to begin") - Cluster.barrier("waiting-for-begin", NrOfNodes).await() - println("Begin!") - - Cluster.barrier("get-ref-to-actor-on-node2", NrOfNodes).await() - - println("Waiting to end") - Cluster.barrier("waiting-to-end", NrOfNodes).await() - println("Shutting down ClusterNode") - Cluster.node.shutdown() - } - } -} */ diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/questions.txt b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/questions.txt deleted file mode 100644 index b02272c30d..0000000000 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/questions.txt +++ /dev/null @@ -1,6 +0,0 @@ -What does clustered home mean? - -akka.actor.deployment.service-hello.clustered.home = "node:node1" - -If a node fails, it should transparently be redeployed on a different node. So actors imho are homeless.. they run -wherever the grid deploys them. \ No newline at end of file diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/testing-design-improvements.txt b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/testing-design-improvements.txt deleted file mode 100644 index 823410999f..0000000000 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin_failover/testing-design-improvements.txt +++ /dev/null @@ -1,54 +0,0 @@ -- It would be nice if the .conf files somehow could be integrated in the scala file - -object SomeNode extends ClusterNodeWithConf{ - def config() = " - akka.event-handler-level = "DEBUG" - akka.actor.deployment.service-hello.router = "round-robin" - akka.actor.deployment.service-hello.clustered.preferred-nodes = ["node:node1"] - akka.actor.deployment.service-hello.clustered.replicas = 1" - } -} - -- It would be nice if the .opts file somehow could be integrated in the scala file. - -object SomeNode extends ClusterNodeWithOpts{ - def opts() = -Dakka.cluster.nodename=node1 -Dakka.cluster.port=9991 -} - -- It should be transparent which node starts/stops the cluster. Perhaps some kind of 'before the world starts' and -'after the world ended' logic could be added. The consequence is that there are mixed responsibilities in a node. - -- A node has the mixed responsibity of being part of the grid and doing checks. It would be nice if one could create -cluster nodes very easily (just spawn a jvm and everything will be copied on them) and if one could create 'client nodes' -that communicate with the grid and do their validations. - -- Each node has been expressed in code, so it is very hard to either use a large number of nodes (lots of code) of to change -the number of nodes without changes all the code. It would be nice if one could say: I want 100 jvm instances with this -specification. - -- There is a lot of waiting for each other, but it would be nice if each node could say this: - waitForGo. - -so you get something like: - -object God{ - def beforeBegin(){ - ZooKeeper.start() - } - - def afterEnd{ - ZooKeeper.stop() - } -} - -class SomeNode extends ClusterTestNode{ - "foo" must { - "bla" in { - waitForGo() - - ..now do testing logic. - } - } -} - - From ef3b82a2c94c3adb3b300960a42d8baca86dd73a Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 2 Aug 2011 15:19:09 +0200 Subject: [PATCH 09/53] Updating Netty to 3.2.5, closing ticket #1074 --- project/AkkaBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 958ad383f5..1c2ddacb61 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -417,7 +417,7 @@ object Dependency { val Jetty = "7.4.0.v20110414" val Logback = "0.9.28" val Multiverse = "0.6.2" - val Netty = "3.2.4.Final" + val Netty = "3.2.5.Final" val Protobuf = "2.4.1" val Scalatest = "1.6.1" val Slf4j = "1.6.0" From a0350d03e91ff138e8cd28734b07c3676e020403 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Tue, 2 Aug 2011 10:19:49 -0600 Subject: [PATCH 10/53] Future: move implicit dispatcher from methods to constructor --- .../src/main/scala/akka/actor/ActorRef.scala | 4 +- .../src/main/scala/akka/dispatch/Future.scala | 78 ++++++++++--------- project/AkkaBuild.scala | 2 +- 3 files changed, 43 insertions(+), 41 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 71b5e7e2d1..32b779c466 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -680,7 +680,7 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, channel: UntypedChannel): Future[Any] = { val future = channel match { case f: ActorPromise ⇒ f - case _ ⇒ new ActorPromise(timeout) + case _ ⇒ new ActorPromise(timeout)(dispatcher) } dispatcher dispatchMessage new MessageInvocation(this, message, future) future @@ -1025,7 +1025,7 @@ private[akka] case class RemoteActorRef private[akka] ( case _ ⇒ None } val future = Actor.remote.send[Any](message, chSender, chFuture, remoteAddress, timeout.duration.toMillis, false, this, loader) - if (future.isDefined) ActorPromise(future.get) + if (future.isDefined) ActorPromise(future.get)(dispatcher) else throw new IllegalActorStateException("Expected a future from remote call to actor " + toString) } diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index b3c2b0c1ed..22bfa963df 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -317,6 +317,8 @@ object Future { sealed trait Future[+T] { + implicit def dispatcher: MessageDispatcher + /** * For use only within a Future.flow block or another compatible Delimited Continuations reset block. * @@ -327,7 +329,7 @@ sealed trait Future[+T] { * execution will fail. The normal result of getting a Future from an ActorRef using ? will return * an untyped Future. */ - def apply[A >: T]()(implicit dispatcher: MessageDispatcher, timeout: Timeout): A @cps[Future[Any]] = shift(this flatMap (_: A ⇒ Future[Any])) + def apply[A >: T]()(implicit timeout: Timeout): A @cps[Future[Any]] = shift(this flatMap (_: A ⇒ Future[Any])) /** * Blocks awaiting completion of this Future, then returns the resulting value, @@ -422,7 +424,7 @@ sealed trait Future[+T] { * Future. If the Future has already been completed, this will apply * immediately. */ - def onComplete(func: Future[T] ⇒ Unit)(implicit dispatcher: MessageDispatcher): this.type + def onComplete(func: Future[T] ⇒ Unit): this.type /** * When the future is completed with a valid result, apply the provided @@ -434,7 +436,7 @@ sealed trait Future[+T] { * } * */ - final def onResult(pf: PartialFunction[Any, Unit])(implicit dispatcher: MessageDispatcher): this.type = onComplete { f ⇒ + final def onResult(pf: PartialFunction[Any, Unit]): this.type = onComplete { f ⇒ val optr = f.result if (optr.isDefined) { val r = optr.get @@ -452,7 +454,7 @@ sealed trait Future[+T] { * } * */ - final def onException(pf: PartialFunction[Throwable, Unit])(implicit dispatcher: MessageDispatcher): Future[T] = onComplete { f ⇒ + final def onException(pf: PartialFunction[Throwable, Unit]): Future[T] = onComplete { f ⇒ val opte = f.exception if (opte.isDefined) { val e = opte.get @@ -460,9 +462,9 @@ sealed trait Future[+T] { } } - def onTimeout(func: Future[T] ⇒ Unit)(implicit dispatcher: MessageDispatcher): this.type + def onTimeout(func: Future[T] ⇒ Unit): this.type - def orElse[A >: T](fallback: ⇒ A)(implicit dispatcher: MessageDispatcher): Future[A] + def orElse[A >: T](fallback: ⇒ A): Future[A] /** * Creates a new Future by applying a PartialFunction to the successful @@ -478,7 +480,7 @@ sealed trait Future[+T] { * } yield b + "-" + c * */ - final def collect[A](pf: PartialFunction[Any, A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Future[A] = { + final def collect[A](pf: PartialFunction[Any, A])(implicit timeout: Timeout): Future[A] = { val future = new DefaultPromise[A](timeout) onComplete { self ⇒ future complete { @@ -510,7 +512,7 @@ sealed trait Future[+T] { * Future(6 / 2) recover { case e: ArithmeticException => 0 } // result: 3 * */ - final def recover[A >: T](pf: PartialFunction[Throwable, A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Future[A] = { + final def recover[A >: T](pf: PartialFunction[Throwable, A])(implicit timeout: Timeout): Future[A] = { val future = new DefaultPromise[A](timeout) onComplete { self ⇒ future complete { @@ -543,7 +545,7 @@ sealed trait Future[+T] { * } yield b + "-" + c * */ - final def map[A](f: T ⇒ A)(implicit dispatcher: MessageDispatcher, timeout: Timeout): Future[A] = { + final def map[A](f: T ⇒ A)(implicit timeout: Timeout): Future[A] = { val future = new DefaultPromise[A](timeout) onComplete { self ⇒ future complete { @@ -567,7 +569,7 @@ sealed trait Future[+T] { * Creates a new Future[A] which is completed with this Future's result if * that conforms to A's erased type or a ClassCastException otherwise. */ - final def mapTo[A](implicit m: Manifest[A], dispatcher: MessageDispatcher = implicitly, timeout: Timeout = this.timeout): Future[A] = { + final def mapTo[A](implicit m: Manifest[A], timeout: Timeout = this.timeout): Future[A] = { val fa = new DefaultPromise[A](timeout) onComplete { ft ⇒ fa complete (ft.value.get match { @@ -597,7 +599,7 @@ sealed trait Future[+T] { * } yield b + "-" + c * */ - final def flatMap[A](f: T ⇒ Future[A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Future[A] = { + final def flatMap[A](f: T ⇒ Future[A])(implicit timeout: Timeout): Future[A] = { val future = new DefaultPromise[A](timeout) onComplete { _.value.get match { @@ -615,23 +617,23 @@ sealed trait Future[+T] { future } - final def foreach(f: T ⇒ Unit)(implicit dispatcher: MessageDispatcher): Unit = onComplete { + final def foreach(f: T ⇒ Unit): Unit = onComplete { _.result match { case Some(v) ⇒ f(v) case None ⇒ } } - final def withFilter(p: T ⇒ Boolean)(implicit dispatcher: MessageDispatcher, timeout: Timeout) = new FutureWithFilter[T](this, p) + final def withFilter(p: T ⇒ Boolean)(implicit timeout: Timeout) = new FutureWithFilter[T](this, p) - final class FutureWithFilter[+A](self: Future[A], p: A ⇒ Boolean)(implicit dispatcher: MessageDispatcher, timeout: Timeout) { + final class FutureWithFilter[+A](self: Future[A], p: A ⇒ Boolean)(implicit timeout: Timeout) { def foreach(f: A ⇒ Unit): Unit = self filter p foreach f def map[B](f: A ⇒ B): Future[B] = self filter p map f def flatMap[B](f: A ⇒ Future[B]): Future[B] = self filter p flatMap f def withFilter(q: A ⇒ Boolean): FutureWithFilter[A] = new FutureWithFilter[A](self, x ⇒ p(x) && q(x)) } - final def filter(p: T ⇒ Boolean)(implicit dispatcher: MessageDispatcher, timeout: Timeout): Future[T] = { + final def filter(p: T ⇒ Boolean)(implicit timeout: Timeout): Future[T] = { val future = new DefaultPromise[T](timeout) onComplete { self ⇒ future complete { @@ -684,17 +686,17 @@ object Promise { /** * Creates a non-completed, new, Promise with the supplied timeout in milliseconds */ - def apply[A](timeout: Timeout): Promise[A] = new DefaultPromise[A](timeout) + def apply[A](timeout: Timeout)(implicit dispatcher: MessageDispatcher): Promise[A] = new DefaultPromise[A](timeout) /** * Creates a non-completed, new, Promise with the default timeout (akka.actor.timeout in conf) */ - def apply[A](): Promise[A] = apply(Timeout.default) + def apply[A]()(implicit dispatcher: MessageDispatcher): Promise[A] = apply(Timeout.default) /** * Construct a completable channel */ - def channel(timeout: Long = Actor.TIMEOUT): ActorPromise = new ActorPromise(timeout) + def channel(timeout: Long = Actor.TIMEOUT)(implicit dispatcher: MessageDispatcher): ActorPromise = new ActorPromise(timeout) private[akka] val callbacksPendingExecution = new ThreadLocal[Option[Stack[() ⇒ Unit]]]() { override def initialValue = None @@ -709,26 +711,26 @@ trait Promise[T] extends Future[T] { * Completes this Future with the specified result, if not already completed. * @return this */ - def complete(value: Either[Throwable, T])(implicit dispatcher: MessageDispatcher): this.type + def complete(value: Either[Throwable, T]): this.type /** * Completes this Future with the specified result, if not already completed. * @return this */ - final def completeWithResult(result: T)(implicit dispatcher: MessageDispatcher): this.type = complete(Right(result)) + final def completeWithResult(result: T): this.type = complete(Right(result)) /** * Completes this Future with the specified exception, if not already completed. * @return this */ - final def completeWithException(exception: Throwable)(implicit dispatcher: MessageDispatcher): this.type = complete(Left(exception)) + final def completeWithException(exception: Throwable): this.type = complete(Left(exception)) /** * Completes this Future with the specified other Future, when that Future is completed, * unless this Future has already been completed. * @return this. */ - final def completeWith(other: Future[T])(implicit dispatcher: MessageDispatcher): this.type = { + final def completeWith(other: Future[T]): this.type = { other onComplete { f ⇒ complete(f.value.get) } this } @@ -768,14 +770,14 @@ trait Promise[T] extends Future[T] { /** * The default concrete Future implementation. */ -class DefaultPromise[T](val timeout: Timeout) extends Promise[T] { +class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDispatcher) extends Promise[T] { self ⇒ - def this() = this(Timeout.default) + def this()(implicit dispatcher: MessageDispatcher) = this(Timeout.default) - def this(timeout: Long) = this(Timeout(timeout)) + def this(timeout: Long)(implicit dispatcher: MessageDispatcher) = this(Timeout(timeout)) - def this(timeout: Long, timeunit: TimeUnit) = this(Timeout(timeout, timeunit)) + def this(timeout: Long, timeunit: TimeUnit)(implicit dispatcher: MessageDispatcher) = this(Timeout(timeout, timeunit)) private val _startTimeInNanos = currentTimeInNanos private val _lock = new ReentrantLock @@ -834,7 +836,7 @@ class DefaultPromise[T](val timeout: Timeout) extends Promise[T] { } } - def complete(value: Either[Throwable, T])(implicit dispatcher: MessageDispatcher): this.type = { + def complete(value: Either[Throwable, T]): this.type = { processCallbacks { _lock.lock try { @@ -858,7 +860,7 @@ class DefaultPromise[T](val timeout: Timeout) extends Promise[T] { this } - private def processCallbacks(callbacks: List[Future[T] ⇒ Unit])(implicit dispatcher: MessageDispatcher): Unit = { + private def processCallbacks(callbacks: List[Future[T] ⇒ Unit]): Unit = { if (callbacks.nonEmpty) { // Steps to ensure we don't run into a stack-overflow situation @tailrec def runCallbacks(rest: List[Future[T] ⇒ Unit], callbackStack: Stack[() ⇒ Unit]) { @@ -887,7 +889,7 @@ class DefaultPromise[T](val timeout: Timeout) extends Promise[T] { } } - def onComplete(func: Future[T] ⇒ Unit)(implicit dispatcher: MessageDispatcher): this.type = { + def onComplete(func: Future[T] ⇒ Unit): this.type = { _lock.lock val notifyNow = try { if (_value.isEmpty) { @@ -905,7 +907,7 @@ class DefaultPromise[T](val timeout: Timeout) extends Promise[T] { this } - def onTimeout(func: Future[T] ⇒ Unit)(implicit dispatcher: MessageDispatcher): this.type = { + def onTimeout(func: Future[T] ⇒ Unit): this.type = { if (timeout.duration.isFinite) { _lock.lock val runNow = try { @@ -930,7 +932,7 @@ class DefaultPromise[T](val timeout: Timeout) extends Promise[T] { this } - final def orElse[A >: T](fallback: ⇒ A)(implicit dispatcher: MessageDispatcher): Future[A] = + final def orElse[A >: T](fallback: ⇒ A): Future[A] = if (timeout.duration.isFinite) { value match { case Some(_) ⇒ this @@ -962,7 +964,7 @@ class DefaultPromise[T](val timeout: Timeout) extends Promise[T] { private def timeLeft(): Long = timeoutInNanos - (currentTimeInNanos - _startTimeInNanos) } -class ActorPromise(timeout: Timeout) extends DefaultPromise[Any](timeout) with ForwardableChannel { +class ActorPromise(timeout: Timeout)(implicit dispatcher: MessageDispatcher) extends DefaultPromise[Any](timeout)(dispatcher) with ForwardableChannel { def !(message: Any)(implicit channel: UntypedChannel = NullChannel) = completeWithResult(message) @@ -980,7 +982,7 @@ class ActorPromise(timeout: Timeout) extends DefaultPromise[Any](timeout) with F } object ActorPromise { - def apply(f: Promise[Any]): ActorPromise = + def apply(f: Promise[Any])(implicit dispatcher: MessageDispatcher): ActorPromise = new ActorPromise(f.timeout) { completeWith(f) override def !(message: Any)(implicit channel: UntypedChannel) = f completeWithResult message @@ -992,11 +994,11 @@ object ActorPromise { * An already completed Future is seeded with it's result at creation, is useful for when you are participating in * a Future-composition but you already have a value to contribute. */ -sealed class KeptPromise[T](suppliedValue: Either[Throwable, T]) extends Promise[T] { +sealed class KeptPromise[T](suppliedValue: Either[Throwable, T])(implicit val dispatcher: MessageDispatcher) extends Promise[T] { val value = Some(suppliedValue) - def complete(value: Either[Throwable, T])(implicit dispatcher: MessageDispatcher): this.type = this - def onComplete(func: Future[T] ⇒ Unit)(implicit dispatcher: MessageDispatcher): this.type = { + def complete(value: Either[Throwable, T]): this.type = this + def onComplete(func: Future[T] ⇒ Unit): this.type = { dispatcher dispatchTask (() ⇒ func(this)) //TODO: Use pending callback stack this } @@ -1005,7 +1007,7 @@ sealed class KeptPromise[T](suppliedValue: Either[Throwable, T]) extends Promise def isExpired: Boolean = true def timeout: Timeout = Timeout.zero - final def onTimeout(func: Future[T] ⇒ Unit)(implicit dispatcher: MessageDispatcher): this.type = this - final def orElse[A >: T](fallback: ⇒ A)(implicit dispatcher: MessageDispatcher): Future[A] = this + final def onTimeout(func: Future[T] ⇒ Unit): this.type = this + final def orElse[A >: T](fallback: ⇒ A): Future[A] = this } diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 5f314004dc..63c6ee0890 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -292,7 +292,7 @@ object AkkaBuild extends Build { resolvers += "Twitter Public Repo" at "http://maven.twttr.com", // This will be going away with com.mongodb.async's next release // compile options - scalacOptions ++= Seq("-encoding", "UTF-8", "-optimise", "-deprecation", "-unchecked"), + scalacOptions ++= Seq("-encoding", "UTF-8", /* "-optimise", */ "-deprecation", "-unchecked"), javacOptions ++= Seq("-Xlint:unchecked", "-Xlint:deprecation"), // add config dir to classpaths From d5683809fed32c7aa7145e836b57842e7beb29fa Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Tue, 2 Aug 2011 11:32:09 -0600 Subject: [PATCH 11/53] revert scalacOptions --- project/AkkaBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 3f39f5be7f..1c2ddacb61 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -292,7 +292,7 @@ object AkkaBuild extends Build { resolvers += "Twitter Public Repo" at "http://maven.twttr.com", // This will be going away with com.mongodb.async's next release // compile options - scalacOptions ++= Seq("-encoding", "UTF-8", /* "-optimise", */ "-deprecation", "-unchecked"), + scalacOptions ++= Seq("-encoding", "UTF-8", "-optimise", "-deprecation", "-unchecked"), javacOptions ++= Seq("-Xlint:unchecked", "-Xlint:deprecation"), // add config dir to classpaths From fbbeacc1e4252fee655b5bcbb234048843b790e4 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Tue, 2 Aug 2011 13:01:10 -0600 Subject: [PATCH 12/53] Allow a Duration to be used with Future.apply --- .../src/test/scala/akka/dispatch/FutureSpec.scala | 5 +++-- akka-actor/src/main/scala/akka/dispatch/Future.scala | 3 +++ 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala index b8e855272a..5a6d7ad214 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala @@ -10,9 +10,10 @@ import org.scalacheck.Arbitrary._ import org.scalacheck.Prop._ import org.scalacheck.Gen._ -import akka.actor.{ Actor, ActorRef } +import akka.actor.{ Actor, ActorRef, Timeout } import Actor._ import akka.testkit.{ EventFilter, filterEvents, filterException } +import akka.util.duration._ import org.multiverse.api.latches.StandardLatch import java.util.concurrent.{ TimeUnit, CountDownLatch } @@ -434,7 +435,7 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd latch.open assert(f2.get === 10) - val f3 = Future({ Thread.sleep(10); 5 }, 10) + val f3 = Future({ Thread.sleep(10); 5 }, 10 millis) filterException[FutureTimeoutException] { intercept[FutureTimeoutException] { f3.get diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 216b3cdf3f..8904eab61d 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -255,6 +255,9 @@ object Future { def apply[T](body: ⇒ T, timeout: Timeout)(implicit dispatcher: MessageDispatcher): Future[T] = apply(body)(dispatcher, timeout) + def apply[T](body: ⇒ T, timeout: Duration)(implicit dispatcher: MessageDispatcher): Future[T] = + apply(body)(dispatcher, timeout) + def apply[T](body: ⇒ T, timeout: Long)(implicit dispatcher: MessageDispatcher): Future[T] = apply(body)(dispatcher, timeout) From 053dbf3b6059168d4669f9388a8ab25b85acd2b1 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Tue, 2 Aug 2011 13:05:55 -0600 Subject: [PATCH 13/53] Update Future docs --- akka-docs/scala/futures.rst | 82 ++++++++++++++++++++----------------- 1 file changed, 45 insertions(+), 37 deletions(-) diff --git a/akka-docs/scala/futures.rst b/akka-docs/scala/futures.rst index e27fd6abfb..4ae489bd22 100644 --- a/akka-docs/scala/futures.rst +++ b/akka-docs/scala/futures.rst @@ -71,26 +71,9 @@ The first method for working with ``Future`` functionally is ``map``. This metho val result = f2.get() -In this example we are joining two strings together within a Future. Instead of waiting for this to complete, we apply our function that calculates the length of the string using the ``map`` method. Now we have a second Future that will eventually contain an ``Int``. When our original ``Future`` completes, it will also apply our function and complete the second Future with it's result. When we finally ``get`` the result, it will contain the number 10. Our original Future still contains the string "HelloWorld" and is unaffected by the ``map``. +In this example we are joining two strings together within a Future. Instead of waiting for this to complete, we apply our function that calculates the length of the string using the ``map`` method. Now we have a second ``Future`` that will eventually contain an ``Int``. When our original ``Future`` completes, it will also apply our function and complete the second ``Future`` with it's result. When we finally ``get`` the result, it will contain the number 10. Our original ``Future`` still contains the string "HelloWorld" and is unaffected by the ``map``. -Something to note when using these methods: if the ``Future`` is still being processed when one of these methods are called, it will be the completing thread that actually does the work. If the ``Future`` is already complete though, it will be run in our current thread. For example: - -.. code-block:: scala - - val f1 = Future { - Thread.sleep(1000) - "Hello" + "World" - } - - val f2 = f1 map { x => - x.length - } - - val result = f2.get() - -The original ``Future`` will take at least 1 second to execute now, which means it is still being processed at the time we call ``map``. The function we provide gets stored within the ``Future`` and later executed automatically by the dispatcher when the result is ready. - -If we do the opposite: +The ``map`` method is fine if we are modifying a single ``Future``, but if 2 or more ``Future``\s are involved ``map`` will not allow you to combine them together: .. code-block:: scala @@ -98,17 +81,19 @@ If we do the opposite: "Hello" + "World" } - Thread.sleep(1000) - - val f2 = f1 map { x => - x.length + val f2 = Future { + 3 } - val result = f2.get() + val f3 = f1 map { x => + f2 map { y => + x.length * y + } + } -Our little string has been processed long before our 1 second sleep has finished. Because of this, the dispatcher has moved onto other messages that need processing and can no longer calculate the length of the string for us, instead it gets calculated in the current thread just as if we weren't using a ``Future``. + val result = f2.get().get() -Normally this works quite well as it means there is very little overhead to running a quick function. If there is a possibility of the function taking a non-trivial amount of time to process it might be better to have this done concurrently, and for that we use ``flatMap``: +The ``get`` method had to be used twice because ``f3`` is a ``Future[Future[Int]]`` instead of the desired ``Future[Int]``. Instead, the ``flatMap`` method should be used: .. code-block:: scala @@ -116,14 +101,18 @@ Normally this works quite well as it means there is very little overhead to runn "Hello" + "World" } - val f2 = f1 flatMap { x => - Future(x.length) + val f2 = Future { + 3 + } + + val f3 = f1 flatMap { x => + f2 map { y => + x.length * y + } } val result = f2.get() -Now our second Future is executed concurrently as well. This technique can also be used to combine the results of several Futures into a single calculation, which will be better explained in the following sections. - For Comprehensions ^^^^^^^^^^^^^^^^^^ @@ -139,7 +128,7 @@ Since ``Future`` has a ``map`` and ``flatMap`` method it can be easily used in a val result = f.get() -Something to keep in mind when doing this is even though it looks like parts of the above example can run in parallel, each step of the for comprehension is run sequentially. This will happen on separate threads for each step but there isn't much benefit over running the calculations all within a single Future. The real benefit comes when the ``Future``\s are created first, and then combining them together. +Something to keep in mind when doing this is even though it looks like parts of the above example can run in parallel, each step of the for comprehension is run sequentially. This will happen on separate threads for each step but there isn't much benefit over running the calculations all within a single ``Future``. The real benefit comes when the ``Future``\s are created first, and then combining them together. Composing Futures ^^^^^^^^^^^^^^^^^ @@ -225,8 +214,6 @@ If the sequence passed to ``fold`` is empty, it will return the start-value, in Same as with ``fold``, the execution will be done by the Thread that completes the last of the Futures, you can also parallize it by chunking your futures into sub-sequences and reduce them, and then reduce the reduced results again. - - This is just a sample of what can be done, but to use more advanced techniques it is easier to take advantage of Scalaz, which Akka has support for in its akka-scalaz module. @@ -257,13 +244,34 @@ In this example, if an ``ArithmeticException`` was thrown while the ``Actor`` pr Timeouts -------- -Waiting forever for a ``Future`` to be completed can be dangerous. It could cause your program to block indefinitly or produce a memory leak. ``Future`` has support for a timeout already builtin with a default of 5 seconds (taken from 'akka.conf'). A timeout is an instance of ``akka.actor.Timeout`` which contains an ``akka.util.Duration``. A ``Duration`` can be finite, which needs a length and unit type, or infinite. An infinite timeout can be dangerous since it will never actually expire. +Waiting forever for a ``Future`` to be completed can be dangerous. It could cause your program to block indefinitly or produce a memory leak. ``Future`` has support for a timeout already builtin with a default of 5 seconds (taken from 'akka.conf'). A timeout is an instance of ``akka.actor.Timeout`` which contains an ``akka.util.Duration``. A ``Duration`` can be finite, which needs a length and unit type, or infinite. An infinite ``Timeout`` can be dangerous since it will never actually expire. -A different timeout can be supplied either explicitly or implicitly when a ``Future`` is created. An implicit timeout has the benefit of being usable by a for-comprehension as well as being picked up by any methods looking for an implicit timeout, while an explicit timeout can be used in a more controlled manner. +A different ``Timeout`` can be supplied either explicitly or implicitly when a ``Future`` is created. An implicit ``Timeout`` has the benefit of being usable by a for-comprehension as well as being picked up by any methods looking for an implicit ``Timeout``, while an explicit ``Timeout`` can be used in a more controlled manner. -Explicit timeout example: +Explicit ``Timeout`` example: -Implicit timeout example: +.. code-block:: scala + + import akka.util.duration._ + + val future1 = Future( { runSomething }, 1 second) + + val future2 = future1.map(doSomethingElse)(1500 millis) + +Implicit ``Timeout`` example: + +.. code-block:: scala + + import akka.actor.Timeout + import akka.util.duration._ + + implicit val longTimeout = Timeout(1 minute) + + val future1 = Future { runSomething } + + val future2 = future1 map doSomethingElse + +An important note: when explicitly providing a ``Timeout`` it is fine to just use a ``Duration`` (like in the above explicit ``Timeout`` example). An implicit ``Duration`` will be ignored if an implicit ``Timeout`` is required. Due to this, in the above implicit example the ``Duration`` is wrapped within a ``Timeout``. If the timeout is reached the ``Future`` becomes unusable, even if an attempt is made to complete it. It is possible to have a ``Future`` handle a timeout, if needed, with the ``onTimeout`` and ``orElse`` methods: From 00494cf95db43bae840324da6d9c4ffb176edfeb Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Tue, 2 Aug 2011 13:38:02 -0600 Subject: [PATCH 14/53] Fix for error while generating scaladocs --- akka-actor/src/main/scala/akka/dispatch/Future.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 8904eab61d..a3cdd9c139 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -310,7 +310,7 @@ object Future { def flow[A](body: ⇒ A @cps[Future[Any]])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Future[A] = { val future = Promise[A](timeout) //dispatcher dispatchTask { () ⇒ - reify(body) foreachFull (future completeWithResult, future completeWithException) onException { + (reify(body) foreachFull (future completeWithResult, future completeWithException): Future[Any]) onException { case e: Exception ⇒ future completeWithException e } //} From 4fac3aa3adc85746914bffed860d91d0aefee1fb Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 3 Aug 2011 10:51:23 +0200 Subject: [PATCH 15/53] Changing initialization and shutdown order to minimize risk of having a stopped actor in the registry --- .../src/main/scala/akka/actor/ActorRef.scala | 28 +++++++++++-------- 1 file changed, 16 insertions(+), 12 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 6f0d1ad2cf..9a8f504dfd 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -554,14 +554,18 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, "Can't restart an actor that has been shut down with 'stop' or 'exit'") if (!isRunning) { dispatcher.attach(this) - _status = ActorRefInternals.RUNNING + try { + // If we are not currently creating this ActorRef instance + if ((actorInstance ne null) && (actorInstance.get ne null)) + initializeActorInstance - // If we are not currently creating this ActorRef instance - if ((actorInstance ne null) && (actorInstance.get ne null)) - initializeActorInstance - - checkReceiveTimeout //Schedule the initial Receive timeout + checkReceiveTimeout //Schedule the initial Receive timeout + } catch { + case e ⇒ + _status = ActorRefInternals.UNSTARTED + throw e + } } this } @@ -574,24 +578,24 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, if (isRunning) { receiveTimeout = None cancelReceiveTimeout - dispatcher.detach(this) + Actor.registry.unregister(this) + + // This lines can trigger cluster start which makes cluster ZK client hang trying to reconnect indefinitely + //if (ClusterModule.isEnabled) Actor.remote.unregister(this) _status = ActorRefInternals.SHUTDOWN + dispatcher.detach(this) try { val a = actor if (Actor.debugLifecycle) EventHandler.debug(a, "stopping") a.postStop() } finally { currentMessage = null - Actor.registry.unregister(this) - - // This lines can trigger cluster start which makes cluster ZK client hang trying to reconnect indefinitely - //if (ClusterModule.isEnabled) Actor.remote.unregister(this) setActorSelfFields(actorInstance.get, null) } } //else if (isBeingRestarted) throw new ActorKilledException("Actor [" + toString + "] is being restarted.") - if (replicationStorage.isDefined) replicationStorage.get.delete() + if (replicationStorage.isDefined) replicationStorage.get.delete() //TODO shouldn't this be inside the if (isRunning?) } } From a589238f9cf699cbe31b28d5d626a26b6d262b0e Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 3 Aug 2011 14:26:01 +0200 Subject: [PATCH 16/53] Set PartialFunction[T,Unit] as onResult callback, closing ticket #1077 --- akka-actor/src/main/scala/akka/dispatch/Future.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index a36c21ee57..c4a21ad8b7 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -419,7 +419,7 @@ sealed trait Future[+T] { * } * */ - final def onResult(pf: PartialFunction[Any, Unit]): this.type = onComplete { f ⇒ + final def onResult(pf: PartialFunction[T, Unit]): this.type = onComplete { f ⇒ val optr = f.result if (optr.isDefined) { val r = optr.get From f70bf9e89cbb00baccb11eee5195c751c141f64b Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 3 Aug 2011 15:01:46 +0200 Subject: [PATCH 17/53] Adding Props to akka.actor package --- .../src/main/scala/akka/actor/ActorRef.scala | 107 +++++++++++++++++- 1 file changed, 106 insertions(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 3af5853eec..80be55d047 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -20,7 +20,8 @@ import java.util.{ Map ⇒ JMap } import scala.reflect.BeanProperty import scala.collection.immutable.Stack import scala.annotation.tailrec -import java.lang.IllegalStateException +import java.lang.{ UnsupportedOperationException, IllegalStateException } +import akka.japi.Creator private[akka] object ActorRefInternals { @@ -39,6 +40,110 @@ private[akka] object ActorRefInternals { } +/** + * ActorRef configuration object, this is threadsafe and fully sharable + * + * Props() returns default configuration + * TODO document me + */ +object Props { + val defaultTimeout: Timeout = Timeout(Duration(Actor.TIMEOUT, "millis")) + def defaultDispatcher: MessageDispatcher = Dispatchers.defaultGlobalDispatcher + val noCreatorSpecified: () ⇒ Actor = () ⇒ throw new UnsupportedOperationException("No actorFactoryProvided!") + + val default = new Props(creator = noCreatorSpecified) + def apply(): Props = default + + def apply[T <: Actor: Manifest]: Props = + default.withCreator(() ⇒ implicitly[Manifest[T]].erasure.asInstanceOf[Class[_ <: Actor]].newInstance) + + def apply(actorClass: Class[_ <: Actor]): Props = + default.withCreator(() ⇒ actorClass.newInstance) +} + +/** + * ActorRef configuration object, this is thread safe and fully sharable + */ +case class Props(creator: () ⇒ Actor, + dispatcher: MessageDispatcher = Props.defaultDispatcher, + timeout: Timeout = Props.defaultTimeout, + receiveTimeout: Option[Duration] = None, + lifeCycle: LifeCycle = Permanent, + faultHandler: FaultHandlingStrategy = NoFaultHandlingStrategy, + supervisor: Option[ActorRef] = None) { + /** + * Returns a new Props with the specified creator set + * Scala API + */ + def withCreator(c: () ⇒ Actor) = copy(creator = c) + + /** + * Returns a new Props with the specified creator set + * Java API + */ + def withCreator(c: Creator[Actor]) = copy(creator = () ⇒ c.create) + + /** + * Returns a new Props with the specified dispatcher set + * Java API + */ + def withDispatcher(d: MessageDispatcher) = copy(dispatcher = d) + + /** + * Returns a new Props with the specified timeout set + * Java API + */ + def withTimeout(t: Timeout) = copy(timeout = t) + + /** + * Returns a new Props with the specified lifecycle set + * Java API + */ + def withLifeCycle(l: LifeCycle) = copy(lifeCycle = l) + + /** + * Returns a new Props with the specified faulthandler set + * Java API + */ + def withFaultHandler(f: FaultHandlingStrategy) = copy(faultHandler = f) + + /** + * Returns a new Props with the specified supervisor set, if null, it's equivalent to withSupervisor(Option.none()) + * Java API + */ + def withSupervisor(s: ActorRef) = copy(supervisor = Option(s)) + + /** + * Returns a new Props with the specified supervisor set + * Java API + */ + def withSupervisor(s: akka.japi.Option[ActorRef]) = copy(supervisor = s.asScala) + + /** + * Returns a new Props with the specified supervisor set + * Scala API + */ + def withSupervisor(s: scala.Option[ActorRef]) = copy(supervisor = s) + + /** + * Returns a new Props with the specified receive timeout set, if null, it's equivalent to withReceiveTimeout(Option.none()) + * Java API + */ + def withReceiveTimeout(r: Duration) = copy(receiveTimeout = Option(r)) + + /** + * Returns a new Props with the specified receive timeout set + * Java API + */ + def withReceiveTimeout(r: akka.japi.Option[Duration]) = copy(receiveTimeout = r.asScala) + + /** + * Returns a new Props with the specified receive timeout set + * Scala API + */ + def withReceiveTimeout(r: scala.Option[Duration]) = copy(receiveTimeout = r) +} + /** * ActorRef is an immutable and serializable handle to an Actor. *

From e565f9cb0407c80d63a22e71a9a1c9663357e720 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 3 Aug 2011 15:43:40 +0200 Subject: [PATCH 18/53] Reenabling mutable zeroes test --- .../test/scala/akka/dispatch/FutureSpec.scala | 25 +++++++++---------- 1 file changed, 12 insertions(+), 13 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala index 5a6d7ad214..ff5199e101 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala @@ -315,21 +315,20 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd } } - /* @Test - def shouldFoldMutableZeroes { - import scala.collection.mutable.ArrayBuffer - def test(testNumber: Int) { - val fs = (0 to 1000) map (i ⇒ Future(i, 10000)) - val result = Futures.fold(ArrayBuffer.empty[AnyRef], 10000)(fs) { - case (l, i) if i % 2 == 0 ⇒ l += i.asInstanceOf[AnyRef] - case (l, _) ⇒ l - }.get.asInstanceOf[ArrayBuffer[Int]].sum + "fold mutable zeroes safely" in { + import scala.collection.mutable.ArrayBuffer + def test(testNumber: Int) { + val fs = (0 to 1000) map (i ⇒ Future(i, 10000)) + val result = Futures.fold(ArrayBuffer.empty[AnyRef], 10000)(fs) { + case (l, i) if i % 2 == 0 ⇒ l += i.asInstanceOf[AnyRef] + case (l, _) ⇒ l + }.get.asInstanceOf[ArrayBuffer[Int]].sum - assert(result === 250500) - } + assert(result === 250500) + } - (1 to 100) foreach test //Make sure it tries to provoke the problem - }*/ + (1 to 100) foreach test //Make sure it tries to provoke the problem + } "return zero value if folding empty list" in { Futures.fold(0)(List[Future[Int]]())(_ + _).get must be(0) From b87b50d9085e87752ec9ba064550b9b8c59558ab Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Wed, 3 Aug 2011 08:42:52 -0600 Subject: [PATCH 19/53] uncomment test --- .../test/scala/akka/dispatch/FutureSpec.scala | 25 +++++++++---------- 1 file changed, 12 insertions(+), 13 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala index 5a6d7ad214..485330c3b4 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala @@ -315,21 +315,20 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd } } - /* @Test - def shouldFoldMutableZeroes { - import scala.collection.mutable.ArrayBuffer - def test(testNumber: Int) { - val fs = (0 to 1000) map (i ⇒ Future(i, 10000)) - val result = Futures.fold(ArrayBuffer.empty[AnyRef], 10000)(fs) { - case (l, i) if i % 2 == 0 ⇒ l += i.asInstanceOf[AnyRef] - case (l, _) ⇒ l - }.get.asInstanceOf[ArrayBuffer[Int]].sum + "shouldFoldMutableZeroes" in { + import scala.collection.mutable.ArrayBuffer + def test(testNumber: Int) { + val fs = (0 to 1000) map (i ⇒ Future(i, 10000)) + val result = Futures.fold(ArrayBuffer.empty[AnyRef], 10000)(fs) { + case (l, i) if i % 2 == 0 ⇒ l += i.asInstanceOf[AnyRef] + case (l, _) ⇒ l + }.get.asInstanceOf[ArrayBuffer[Int]].sum - assert(result === 250500) - } + assert(result === 250500) + } - (1 to 100) foreach test //Make sure it tries to provoke the problem - }*/ + (1 to 100) foreach test //Make sure it tries to provoke the problem + } "return zero value if folding empty list" in { Futures.fold(0)(List[Future[Int]]())(_ + _).get must be(0) From d37881841403ad0c5e73c7f6294d51623b79d998 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Wed, 3 Aug 2011 10:11:24 -0600 Subject: [PATCH 20/53] Use dispatcher from the passed in Future --- akka-actor/src/main/scala/akka/actor/ActorRef.scala | 2 +- akka-actor/src/main/scala/akka/dispatch/Future.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 80be55d047..55b8a4e4d5 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -1136,7 +1136,7 @@ private[akka] case class RemoteActorRef private[akka] ( case _ ⇒ None } val future = Actor.remote.send[Any](message, chSender, chFuture, remoteAddress, timeout.duration.toMillis, false, this, loader) - if (future.isDefined) ActorPromise(future.get)(dispatcher) + if (future.isDefined) ActorPromise(future.get) else throw new IllegalActorStateException("Expected a future from remote call to actor " + toString) } diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 4f8a47ae01..24ea405f4f 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -983,8 +983,8 @@ class ActorPromise(timeout: Timeout)(implicit dispatcher: MessageDispatcher) ext } object ActorPromise { - def apply(f: Promise[Any])(implicit dispatcher: MessageDispatcher): ActorPromise = - new ActorPromise(f.timeout) { + def apply(f: Promise[Any]): ActorPromise = + new ActorPromise(f.timeout)(f.dispatcher) { completeWith(f) override def !(message: Any)(implicit channel: UntypedChannel) = f completeWithResult message override def sendException(ex: Throwable) = f completeWithException ex From d67fe8befea44e7ee3ed0d2a930a8c98820480ac Mon Sep 17 00:00:00 2001 From: Peter Veentjer Date: Thu, 4 Aug 2011 07:22:16 +0300 Subject: [PATCH 21/53] ticket #1032 --- .../akka/cluster/TransactionLogSpec.scala | 33 +++++++++---------- 1 file changed, 16 insertions(+), 17 deletions(-) diff --git a/akka-cluster/src/test/scala/akka/cluster/TransactionLogSpec.scala b/akka-cluster/src/test/scala/akka/cluster/TransactionLogSpec.scala index 259c5179e1..71afeff1e9 100644 --- a/akka-cluster/src/test/scala/akka/cluster/TransactionLogSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/TransactionLogSpec.scala @@ -28,16 +28,15 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA private var bookKeeper: BookKeeper = _ private var localBookKeeper: LocalBookKeeper = _ - // synchronous API - "A Transaction Log" should { - "be able to record entries - synchronous" in { + "A synchronous used Transaction Log" should { + "be able to record entries" in { val uuid = (new UUID).toString val txlog = TransactionLog.newLogFor(uuid, false, null) val entry = "hello".getBytes("UTF-8") txlog.recordEntry(entry) } - "be able to record and delete entries - synchronous" in { + "be able to record and delete entries" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, false, null) val entry = "hello".getBytes("UTF-8") @@ -48,7 +47,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA intercept[BKNoSuchLedgerExistsException](TransactionLog.logFor(uuid, false, null)) } - "be able to record entries and read entries with 'entriesInRange' - synchronous" in { + "be able to record entries and read entries with 'entriesInRange'" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, false, null) val entry = "hello".getBytes("UTF-8") @@ -64,7 +63,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA txlog2.close } - "be able to record entries and read entries with 'entries' - synchronous" in { + "be able to record entries and read entries with 'entries'" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, false, null) val entry = "hello".getBytes("UTF-8") @@ -84,7 +83,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA txlog2.close } - "be able to record a snapshot - synchronous" in { + "be able to record a snapshot" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, false, null) val snapshot = "snapshot".getBytes("UTF-8") @@ -92,7 +91,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA txlog1.close } - "be able to record and read a snapshot and following entries - synchronous" in { + "be able to record and read a snapshot and following entries" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, false, null) val snapshot = "snapshot".getBytes("UTF-8") @@ -118,7 +117,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA txlog2.close } - "be able to record entries then a snapshot then more entries - and then read from the snapshot and the following entries - synchronous" in { + "be able to record entries then a snapshot then more entries - and then read from the snapshot and the following entries" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, false, null) @@ -146,8 +145,8 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA } } - "A Transaction Log" should { - "be able to record entries - asynchronous" in { + "An asynchronous used Transaction Log" should { + "be able to record entries" in { val uuid = (new UUID).toString val txlog = TransactionLog.newLogFor(uuid, true, null) val entry = "hello".getBytes("UTF-8") @@ -156,7 +155,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA txlog.close } - "be able to record and delete entries - asynchronous" in { + "be able to record and delete entries" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, true, null) Thread.sleep(200) @@ -169,7 +168,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA Thread.sleep(200) intercept[BKNoSuchLedgerExistsException](TransactionLog.logFor(uuid, true, null)) } - "be able to record entries and read entries with 'entriesInRange' - asynchronous" in { + "be able to record entries and read entries with 'entriesInRange'" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, true, null) Thread.sleep(200) @@ -191,7 +190,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA txlog2.close } - "be able to record entries and read entries with 'entries' - asynchronous" in { + "be able to record entries and read entries with 'entries'" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, true, null) Thread.sleep(200) @@ -218,7 +217,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA txlog2.close } - "be able to record a snapshot - asynchronous" in { + "be able to record a snapshot" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, true, null) Thread.sleep(200) @@ -228,7 +227,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA txlog1.close } - "be able to record and read a snapshot and following entries - asynchronous" in { + "be able to record and read a snapshot and following entries" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, true, null) Thread.sleep(200) @@ -264,7 +263,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA txlog2.close } - "be able to record entries then a snapshot then more entries - and then read from the snapshot and the following entries - asynchronous" in { + "be able to record entries then a snapshot then more entries - and then read from the snapshot and the following entries" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, true, null) Thread.sleep(200) From ea369a4b65f23b53fb403c79b21d65a399f52c0d Mon Sep 17 00:00:00 2001 From: Peter Vlugter Date: Thu, 4 Aug 2011 16:49:05 +1200 Subject: [PATCH 22/53] Quieten the multi-jvm tests --- .../scala/akka/cluster/ClusterTestNode.scala | 4 --- .../scala/akka/cluster/QuietReporter.scala | 20 +++++++++++ .../ClusterActorRefCleanupMultiJvmNode1.conf | 1 + .../ClusterActorRefCleanupMultiJvmSpec.scala | 27 ++++++++++----- ...LogWriteBehindNoSnapshotMultiJvmNode2.conf | 2 +- ...onLogWriteBehindSnapshotMultiJvmNode2.conf | 2 +- ...onLogWriteBehindSnapshotMultiJvmSpec.scala | 4 +-- ...ogWriteThroughNoSnapshotMultiJvmNode2.conf | 2 +- ...nLogWriteThroughSnapshotMultiJvmNode2.conf | 2 +- ...nLogWriteThroughSnapshotMultiJvmSpec.scala | 4 +-- .../FailoverDirectRoutingMultiJvmNode1.conf | 1 + .../FailoverDirectRoutingMultiJvmSpec.scala | 30 +++++++++-------- ...ngleReplicaDirectRoutingMultiJvmSpec.scala | 8 ++--- .../failover/RandomFailoverMultiJvmNode1.conf | 1 + .../failover/RandomFailoverMultiJvmSpec.scala | 29 +++++++++------- .../RoundRobinFailoverMultiJvmNode1.conf | 1 + .../RoundRobinFailoverMultiJvmSpec.scala | 33 ++++++++++++------- .../src/test/resources/log4j.properties | 2 +- project/AkkaBuild.scala | 9 +++-- project/plugins/build.sbt | 2 +- 20 files changed, 120 insertions(+), 64 deletions(-) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/QuietReporter.scala diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterTestNode.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterTestNode.scala index bdc430ee6d..302fa52058 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterTestNode.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterTestNode.scala @@ -55,7 +55,6 @@ object ClusterTestNode { val Timeout = 1.minute def ready(className: String) = { - println("ClusterTest: READY") readyFile(className).createNewFile() } @@ -64,11 +63,9 @@ object ClusterTestNode { cleanUp(className) sys.error("Timeout waiting for cluster ready") } - println("ClusterTest: GO") } def exit(className: String) = { - println("ClusterTest: EXIT") exitFile(className).createNewFile() } @@ -77,7 +74,6 @@ object ClusterTestNode { cleanUp(className) sys.error("Timeout waiting for node exits") } - println("ClusterTest: SHUTDOWN") } def cleanUp(className: String) = { diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/QuietReporter.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/QuietReporter.scala new file mode 100644 index 0000000000..005b423b04 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/QuietReporter.scala @@ -0,0 +1,20 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ + +package org.scalatest.akka + +import org.scalatest.tools.StandardOutReporter +import org.scalatest.events._ +import java.lang.Boolean.getBoolean + +class QuietReporter(inColor: Boolean) extends StandardOutReporter(false, inColor, false, false) { + def this() = this(!getBoolean("akka.test.nocolor")) + + override def apply(event: Event): Unit = event match { + case _: RunStarting ⇒ () + case _ ⇒ super.apply(event) + } + + override def makeFinalReport(resourceName: String, duration: Option[Long], summaryOption: Option[Summary]): Unit = {} +} diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode1.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode1.conf index 518aed1cd0..65bd1e0e07 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode1.conf +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmNode1.conf @@ -1,4 +1,5 @@ akka.enabled-modules = ["cluster"] +akka.event-handlers = ["akka.testkit.TestEventListener"] akka.event-handler-level = "ERROR" akka.actor.deployment.service-test.router = "round-robin" akka.actor.deployment.service-test.clustered.preferred-nodes = ["node:node2","node:node3"] diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala index 7c59a9fb93..fc4b2b3f5e 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala @@ -7,7 +7,10 @@ package akka.cluster.reflogic import akka.cluster._ import akka.cluster.Cluster._ import akka.actor.Actor +import akka.event.EventHandler +import akka.testkit.{ EventFilter, TestEvent } import akka.routing.RoutingException +import java.net.ConnectException import java.nio.channels.{ ClosedChannelException, NotYetConnectedException } object ClusterActorRefCleanupMultiJvmSpec { @@ -15,16 +18,16 @@ object ClusterActorRefCleanupMultiJvmSpec { val NrOfNodes = 3 class TestActor extends Actor with Serializable { - println("--------------------------------------") - println("TestActor created") - println("--------------------------------------") + //println("--------------------------------------") + //println("TestActor created") + //println("--------------------------------------") def receive = { case "Die" ⇒ - println("Killing JVM: " + Cluster.node.nodeAddress) + //println("Killing JVM: " + Cluster.node.nodeAddress) System.exit(0) case _ ⇒ - println("Hello") + //println("Hello") } } @@ -50,6 +53,15 @@ class ClusterActorRefCleanupMultiJvmNode1 extends MasterClusterTestNode { //verify that all remote actors are there. clusteredRef.connections.size must be(2) + // ignore exceptions from killing nodes + val ignoreExceptions = Seq( + EventFilter[ClosedChannelException], + EventFilter[NotYetConnectedException], + EventFilter[RoutingException], + EventFilter[ConnectException]) + + EventHandler.notify(TestEvent.Mute(ignoreExceptions)) + //let one of the actors die. clusteredRef ! "Die" @@ -89,11 +101,8 @@ class ClusterActorRefCleanupMultiJvmNode1 extends MasterClusterTestNode { clusteredRef.connections.size must be(0) //and lets make sure we now get the correct exception if we try to use the ref. - try { + intercept[RoutingException] { clusteredRef ! "Hello" - assert(false) - } catch { - case e: RoutingException ⇒ } node.shutdown() diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmNode2.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmNode2.conf index 370167dcad..7e6054a1f8 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmNode2.conf +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmNode2.conf @@ -1,5 +1,5 @@ akka.enabled-modules = ["cluster"] -akka.event-handler-level = "WARNING" +akka.event-handler-level = "ERROR" akka.actor.deployment.hello-world.router = "direct" akka.actor.deployment.hello-world.clustered.replication-factor = 1 akka.actor.deployment.hello-world.clustered.replication.storage = "transaction-log" diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/snapshot/ReplicationTransactionLogWriteBehindSnapshotMultiJvmNode2.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/snapshot/ReplicationTransactionLogWriteBehindSnapshotMultiJvmNode2.conf index 97a0fb3687..a118d2fcf2 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/snapshot/ReplicationTransactionLogWriteBehindSnapshotMultiJvmNode2.conf +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/snapshot/ReplicationTransactionLogWriteBehindSnapshotMultiJvmNode2.conf @@ -1,5 +1,5 @@ akka.enabled-modules = ["cluster"] -akka.event-handler-level = "WARNING" +akka.event-handler-level = "ERROR" akka.actor.deployment.hello-world.router = "direct" akka.actor.deployment.hello-world.clustered.replication-factor = 1 akka.actor.deployment.hello-world.clustered.replication.storage = "transaction-log" diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/snapshot/ReplicationTransactionLogWriteBehindSnapshotMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/snapshot/ReplicationTransactionLogWriteBehindSnapshotMultiJvmSpec.scala index 80ad04a4df..a186fefa85 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/snapshot/ReplicationTransactionLogWriteBehindSnapshotMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/snapshot/ReplicationTransactionLogWriteBehindSnapshotMultiJvmSpec.scala @@ -19,11 +19,11 @@ object ReplicationTransactionLogWriteBehindSnapshotMultiJvmSpec { class HelloWorld extends Actor with Serializable { var log = "" - println("Creating HelloWorld log =======> " + log) + //println("Creating HelloWorld log =======> " + log) def receive = { case Count(nr) ⇒ log += nr.toString - println("Message to HelloWorld log =======> " + log) + //println("Message to HelloWorld log =======> " + log) self.reply("World from node [" + Config.nodename + "]") case GetLog ⇒ self.reply(Log(log)) diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmNode2.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmNode2.conf index cc2fb1ef3b..45bbc96e4d 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmNode2.conf +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmNode2.conf @@ -1,5 +1,5 @@ akka.enabled-modules = ["cluster"] -akka.event-handler-level = "WARNING" +akka.event-handler-level = "ERROR" akka.actor.deployment.hello-world.router = "direct" akka.actor.deployment.hello-world.clustered.replication-factor = 1 akka.actor.deployment.hello-world.clustered.replication.storage = "transaction-log" diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/snapshot/ReplicationTransactionLogWriteThroughSnapshotMultiJvmNode2.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/snapshot/ReplicationTransactionLogWriteThroughSnapshotMultiJvmNode2.conf index c38191ce28..fac120a46f 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/snapshot/ReplicationTransactionLogWriteThroughSnapshotMultiJvmNode2.conf +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/snapshot/ReplicationTransactionLogWriteThroughSnapshotMultiJvmNode2.conf @@ -1,5 +1,5 @@ akka.enabled-modules = ["cluster"] -akka.event-handler-level = "WARNING" +akka.event-handler-level = "ERROR" akka.actor.deployment.hello-world.router = "direct" akka.actor.deployment.hello-world.clustered.replication-factor = 1 akka.actor.deployment.hello-world.clustered.replication.storage = "transaction-log" diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/snapshot/ReplicationTransactionLogWriteThroughSnapshotMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/snapshot/ReplicationTransactionLogWriteThroughSnapshotMultiJvmSpec.scala index b1136a5490..45418350d0 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/snapshot/ReplicationTransactionLogWriteThroughSnapshotMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/snapshot/ReplicationTransactionLogWriteThroughSnapshotMultiJvmSpec.scala @@ -19,11 +19,11 @@ object ReplicationTransactionLogWriteThroughSnapshotMultiJvmSpec { class HelloWorld extends Actor with Serializable { var log = "" - println("Creating HelloWorld log =======> " + log) + //println("Creating HelloWorld log =======> " + log) def receive = { case Count(nr) ⇒ log += nr.toString - println("Message to HelloWorld log =======> " + log) + //println("Message to HelloWorld log =======> " + log) self.reply("World from node [" + Config.nodename + "]") case GetLog ⇒ self.reply(Log(log)) diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/failover/FailoverDirectRoutingMultiJvmNode1.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/failover/FailoverDirectRoutingMultiJvmNode1.conf index 602bc41489..de758ebbba 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/failover/FailoverDirectRoutingMultiJvmNode1.conf +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/failover/FailoverDirectRoutingMultiJvmNode1.conf @@ -1,4 +1,5 @@ akka.enabled-modules = ["cluster"] +akka.event-handlers = ["akka.testkit.TestEventListener"] akka.event-handler-level = "WARNING" akka.actor.deployment.service-hello.router = "direct" akka.actor.deployment.service-hello.clustered.home = "node:node2" \ No newline at end of file diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/failover/FailoverDirectRoutingMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/failover/FailoverDirectRoutingMultiJvmSpec.scala index df41e93fd9..b0914cf24d 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/failover/FailoverDirectRoutingMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/failover/FailoverDirectRoutingMultiJvmSpec.scala @@ -4,22 +4,26 @@ import akka.config.Config import scala.Predef._ import akka.cluster.{ ClusterActorRef, Cluster, MasterClusterTestNode, ClusterTestNode } import akka.actor.{ ActorInitializationException, Actor } +import akka.event.EventHandler +import akka.testkit.{ EventFilter, TestEvent } +import java.net.ConnectException +import java.nio.channels.NotYetConnectedException object FailoverDirectRoutingMultiJvmSpec { val NrOfNodes = 2 class SomeActor extends Actor with Serializable { - println("---------------------------------------------------------------------------") - println("SomeActor has been created on node [" + Config.nodename + "]") - println("---------------------------------------------------------------------------") + //println("---------------------------------------------------------------------------") + //println("SomeActor has been created on node [" + Config.nodename + "]") + //println("---------------------------------------------------------------------------") def receive = { case "identify" ⇒ - println("The node received the 'identify' command: " + Config.nodename) + //println("The node received the 'identify' command: " + Config.nodename) self.reply(Config.nodename) case "die" ⇒ - println("The node received the 'die' command: " + Config.nodename) + //println("The node received the 'die' command: " + Config.nodename) Cluster.node.shutdown } } @@ -35,27 +39,27 @@ class FailoverDirectRoutingMultiJvmNode1 extends MasterClusterTestNode { "Direct Router" must { "not yet be able to failover to another node" in { - println("==================================================================================================") - println(" FAILOVER DIRECT ROUTING") - println("==================================================================================================") + //println("==================================================================================================") + //println(" FAILOVER DIRECT ROUTING") + //println("==================================================================================================") + + val ignoreExceptions = Seq(EventFilter[NotYetConnectedException], EventFilter[ConnectException]) + EventHandler.notify(TestEvent.Mute(ignoreExceptions)) Cluster.node.start() Cluster.barrier("waiting-for-begin", NrOfNodes).await() val actor = Actor.actorOf[SomeActor]("service-hello").start().asInstanceOf[ClusterActorRef] - println("retrieved identity was: " + (actor ? "identify").get) + //println("retrieved identity was: " + (actor ? "identify").get) (actor ? "identify").get must equal("node2") actor ! "die" Thread.sleep(4000) - try { + intercept[ActorInitializationException] { actor ! "identify" - fail() - } catch { - case e: ActorInitializationException ⇒ } } } diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/normalusage/SingleReplicaDirectRoutingMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/normalusage/SingleReplicaDirectRoutingMultiJvmSpec.scala index 9bc9681263..793c3cbdd0 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/normalusage/SingleReplicaDirectRoutingMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/direct/normalusage/SingleReplicaDirectRoutingMultiJvmSpec.scala @@ -8,13 +8,13 @@ object SingleReplicaDirectRoutingMultiJvmSpec { val NrOfNodes = 2 class SomeActor extends Actor with Serializable { - println("---------------------------------------------------------------------------") - println("SomeActor has been created on node [" + Config.nodename + "]") - println("---------------------------------------------------------------------------") + //println("---------------------------------------------------------------------------") + //println("SomeActor has been created on node [" + Config.nodename + "]") + //println("---------------------------------------------------------------------------") def receive = { case "identify" ⇒ { - println("The node received the 'identify' command: " + Config.nodename) + //println("The node received the 'identify' command: " + Config.nodename) self.reply(Config.nodename) } } diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmNode1.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmNode1.conf index c75c38be44..848ef3f7d3 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmNode1.conf +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmNode1.conf @@ -1,4 +1,5 @@ akka.enabled-modules = ["cluster"] +akka.event-handlers = ["akka.testkit.TestEventListener"] akka.event-handler-level = "WARNING" akka.actor.deployment.service-hello.router = "random" akka.actor.deployment.service-hello.clustered.preferred-nodes = ["node:node2"] diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmSpec.scala index 27dc35a8f6..756d3c7e6b 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/failover/RandomFailoverMultiJvmSpec.scala @@ -3,24 +3,28 @@ package akka.cluster.routing.random.failover import akka.config.Config import akka.cluster._ import akka.actor.{ ActorRef, Actor } +import akka.event.EventHandler +import akka.testkit.{ EventFilter, TestEvent } import java.util.{ Collections, Set ⇒ JSet } +import java.net.ConnectException +import java.nio.channels.NotYetConnectedException object RandomFailoverMultiJvmSpec { val NrOfNodes = 3 class SomeActor extends Actor with Serializable { - println("---------------------------------------------------------------------------") - println("SomeActor has been created on node [" + Config.nodename + "]") - println("---------------------------------------------------------------------------") + //println("---------------------------------------------------------------------------") + //println("SomeActor has been created on node [" + Config.nodename + "]") + //println("---------------------------------------------------------------------------") def receive = { case "identify" ⇒ { - println("The node received the 'identify' command") + //println("The node received the 'identify' command") self.reply(Config.nodename) } case "shutdown" ⇒ { - println("The node received the 'shutdown' command") + //println("The node received the 'shutdown' command") Cluster.node.shutdown() } } @@ -35,13 +39,16 @@ class RandomFailoverMultiJvmNode1 extends MasterClusterTestNode { def testNodes = NrOfNodes def sleepSome() { - println("Starting sleep") + //println("Starting sleep") Thread.sleep(1000) //nasty.. but ok for now. - println("Finished doing sleep") + //println("Finished doing sleep") } "Random: when routing fails" must { "jump to another replica" in { + val ignoreExceptions = Seq(EventFilter[NotYetConnectedException], EventFilter[ConnectException]) + EventHandler.notify(TestEvent.Mute(ignoreExceptions)) + Cluster.node.start() Cluster.barrier("waiting-for-begin", NrOfNodes).await() @@ -49,8 +56,8 @@ class RandomFailoverMultiJvmNode1 extends MasterClusterTestNode { val actor = Actor.actorOf[SomeActor]("service-hello").asInstanceOf[ClusterActorRef] val oldFoundConnections = identifyConnections(actor) - println("---------------------------- oldFoundConnections ------------------------") - println(oldFoundConnections) + //println("---------------------------- oldFoundConnections ------------------------") + //println(oldFoundConnections) //since we have replication factor 2 oldFoundConnections.size() must be(2) @@ -65,8 +72,8 @@ class RandomFailoverMultiJvmNode1 extends MasterClusterTestNode { //the test code has been deactivated to prevent causing problems. val newFoundConnections = identifyConnections(actor) - println("---------------------------- newFoundConnections ------------------------") - println(newFoundConnections) + //println("---------------------------- newFoundConnections ------------------------") + //println(newFoundConnections) //it still must be 2 since a different node should have been used to failover to newFoundConnections.size() must be(2) diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmNode1.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmNode1.conf index f70ecc2a65..095ef7c0d5 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmNode1.conf +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmNode1.conf @@ -1,4 +1,5 @@ akka.enabled-modules = ["cluster"] +akka.event-handlers = ["akka.testkit.TestEventListener"] akka.event-handler-level = "WARNING" akka.actor.deployment.service-hello.router = "round-robin" akka.actor.deployment.service-hello.clustered.preferred-nodes = ["node:node2"] diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmSpec.scala index f8c5fc0380..665c839c28 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/failover/RoundRobinFailoverMultiJvmSpec.scala @@ -3,24 +3,28 @@ package akka.cluster.routing.roundrobin.failover import akka.config.Config import akka.cluster._ import akka.actor.{ ActorRef, Actor } +import akka.event.EventHandler +import akka.testkit.{ EventFilter, TestEvent } import java.util.{ Collections, Set ⇒ JSet } +import java.net.ConnectException +import java.nio.channels.NotYetConnectedException object RoundRobinFailoverMultiJvmSpec { val NrOfNodes = 3 class SomeActor extends Actor with Serializable { - println("---------------------------------------------------------------------------") - println("SomeActor has been created on node [" + Config.nodename + "]") - println("---------------------------------------------------------------------------") + //println("---------------------------------------------------------------------------") + //println("SomeActor has been created on node [" + Config.nodename + "]") + //println("---------------------------------------------------------------------------") def receive = { case "identify" ⇒ { - println("The node received the 'identify' command") + //println("The node received the 'identify' command") self.reply(Config.nodename) } case "shutdown" ⇒ { - println("The node received the 'shutdown' command") + //println("The node received the 'shutdown' command") Cluster.node.shutdown() } } @@ -35,13 +39,20 @@ class RoundRobinFailoverMultiJvmNode1 extends MasterClusterTestNode { def testNodes = NrOfNodes def sleepSome() { - println("Starting sleep") + //println("Starting sleep") Thread.sleep(1000) //nasty.. but ok for now. - println("Finished doing sleep") + //println("Finished doing sleep") } "Round Robin: when round robin fails" must { "jump to another replica" in { + val ignoreExceptions = Seq( + EventFilter[NotYetConnectedException], + EventFilter[ConnectException], + EventFilter[ClusterException]) + + EventHandler.notify(TestEvent.Mute(ignoreExceptions)) + Cluster.node.start() Cluster.barrier("waiting-for-begin", NrOfNodes).await() @@ -49,8 +60,8 @@ class RoundRobinFailoverMultiJvmNode1 extends MasterClusterTestNode { val actor = Actor.actorOf[SomeActor]("service-hello").asInstanceOf[ClusterActorRef] val oldFoundConnections = identifyConnections(actor) - println("---------------------------- oldFoundConnections ------------------------") - println(oldFoundConnections) + //println("---------------------------- oldFoundConnections ------------------------") + //println(oldFoundConnections) //since we have replication factor 2 oldFoundConnections.size() must be(2) @@ -66,8 +77,8 @@ class RoundRobinFailoverMultiJvmNode1 extends MasterClusterTestNode { /* val newFoundConnections = identifyConnections(actor) - println("---------------------------- newFoundConnections ------------------------") - println(newFoundConnections) + //println("---------------------------- newFoundConnections ------------------------") + //println(newFoundConnections) //it still must be 2 since a different node should have been used to failover to newFoundConnections.size() must be(2) diff --git a/akka-cluster/src/test/resources/log4j.properties b/akka-cluster/src/test/resources/log4j.properties index 9825970594..2d07c8e051 100644 --- a/akka-cluster/src/test/resources/log4j.properties +++ b/akka-cluster/src/test/resources/log4j.properties @@ -1,6 +1,6 @@ # Define some default values that can be overridden by system properties zookeeper.root.logger=INFO, CONSOLE -zookeeper.console.threshold=INFO +zookeeper.console.threshold=OFF zookeeper.log.dir=. zookeeper.log.file=zookeeper.log zookeeper.log.threshold=DEBUG diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 1c2ddacb61..a997e6d7ab 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -1,7 +1,8 @@ import sbt._ import Keys._ -import MultiJvmPlugin.{ MultiJvm, extraOptions } +import MultiJvmPlugin.{ MultiJvm, extraOptions, jvmOptions, scalatestOptions } import ScalariformPlugin.{ format, formatPreferences } +import java.lang.Boolean.getBoolean object AkkaBuild extends Build { System.setProperty("akka.mode", "test") // Is there better place for this? @@ -71,6 +72,10 @@ object AkkaBuild extends Build { extraOptions in MultiJvm <<= (sourceDirectory in MultiJvm) { src => (name: String) => (src ** (name + ".conf")).get.headOption.map("-Dakka.config=" + _.absolutePath).toSeq }, + scalatestOptions in MultiJvm := Seq("-r", "org.scalatest.akka.QuietReporter"), + jvmOptions in MultiJvm := Seq("-Xmx256") ++ { + if (getBoolean("sbt.log.noformat")) Seq("-Dakka.test.nocolor=true") else Nil + }, test in Test <<= (test in Test) dependsOn (test in MultiJvm) ) ) configs (MultiJvm) @@ -456,7 +461,7 @@ object Dependency { val springBeans = "org.springframework" % "spring-beans" % V.Spring // ApacheV2 val springContext = "org.springframework" % "spring-context" % V.Spring // ApacheV2 val staxApi = "javax.xml.stream" % "stax-api" % "1.0-2" // ApacheV2 - val twttrUtilCore = "com.twitter" % "util-core" % "1.8.1" // ApacheV2 + val twttrUtilCore = "com.twitter" % "util-core" % "1.8.1" // ApacheV2 val zkClient = "zkclient" % "zkclient" % "0.3" // ApacheV2 val zookeeper = "org.apache.hadoop.zookeeper" % "zookeeper" % V.Zookeeper // ApacheV2 val zookeeperLock = "org.apache.hadoop.zookeeper" % "zookeeper-recipes-lock" % V.Zookeeper // ApacheV2 diff --git a/project/plugins/build.sbt b/project/plugins/build.sbt index 763dd19166..8db73933f3 100644 --- a/project/plugins/build.sbt +++ b/project/plugins/build.sbt @@ -1,6 +1,6 @@ resolvers += "Typesafe Repo" at "http://repo.typesafe.com/typesafe/releases/" -libraryDependencies += "com.typesafe.sbt-multi-jvm" %% "sbt-multi-jvm" % "0.1.3" +libraryDependencies += "com.typesafe.sbt-multi-jvm" %% "sbt-multi-jvm" % "0.1.4" libraryDependencies += "com.typesafe.sbt-scalariform" %% "sbt-scalariform" % "0.1.2" From b41778fbc4c2bccc59b67df929ba892050e65ea9 Mon Sep 17 00:00:00 2001 From: Peter Vlugter Date: Thu, 4 Aug 2011 17:22:34 +1200 Subject: [PATCH 23/53] Remove heap size option for multi-jvm --- project/AkkaBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index a997e6d7ab..84a16ef0cf 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -73,7 +73,7 @@ object AkkaBuild extends Build { (name: String) => (src ** (name + ".conf")).get.headOption.map("-Dakka.config=" + _.absolutePath).toSeq }, scalatestOptions in MultiJvm := Seq("-r", "org.scalatest.akka.QuietReporter"), - jvmOptions in MultiJvm := Seq("-Xmx256") ++ { + jvmOptions in MultiJvm := { if (getBoolean("sbt.log.noformat")) Seq("-Dakka.test.nocolor=true") else Nil }, test in Test <<= (test in Test) dependsOn (test in MultiJvm) From b66ebdcc7258dfc0da5de8ec269a75a16ac13297 Mon Sep 17 00:00:00 2001 From: Peter Veentjer Date: Fri, 5 Aug 2011 05:56:15 +0300 Subject: [PATCH 24/53] ticket #1032.. more cleanup --- .../akka/cluster/TransactionLogSpec.scala | 27 ++++++++++--------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/akka-cluster/src/test/scala/akka/cluster/TransactionLogSpec.scala b/akka-cluster/src/test/scala/akka/cluster/TransactionLogSpec.scala index 71afeff1e9..291e5a995b 100644 --- a/akka-cluster/src/test/scala/akka/cluster/TransactionLogSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/TransactionLogSpec.scala @@ -29,14 +29,15 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA private var localBookKeeper: LocalBookKeeper = _ "A synchronous used Transaction Log" should { - "be able to record entries" in { + + "be able to record entries - synchronous" in { val uuid = (new UUID).toString val txlog = TransactionLog.newLogFor(uuid, false, null) val entry = "hello".getBytes("UTF-8") txlog.recordEntry(entry) } - "be able to record and delete entries" in { + "be able to record and delete entries - synchronous" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, false, null) val entry = "hello".getBytes("UTF-8") @@ -47,7 +48,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA intercept[BKNoSuchLedgerExistsException](TransactionLog.logFor(uuid, false, null)) } - "be able to record entries and read entries with 'entriesInRange'" in { + "be able to record entries and read entries with 'entriesInRange' - synchronous" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, false, null) val entry = "hello".getBytes("UTF-8") @@ -63,7 +64,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA txlog2.close } - "be able to record entries and read entries with 'entries'" in { + "be able to record entries and read entries with 'entries' - synchronous" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, false, null) val entry = "hello".getBytes("UTF-8") @@ -83,7 +84,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA txlog2.close } - "be able to record a snapshot" in { + "be able to record a snapshot - synchronous" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, false, null) val snapshot = "snapshot".getBytes("UTF-8") @@ -91,7 +92,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA txlog1.close } - "be able to record and read a snapshot and following entries" in { + "be able to record and read a snapshot and following entries - synchronous" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, false, null) val snapshot = "snapshot".getBytes("UTF-8") @@ -117,7 +118,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA txlog2.close } - "be able to record entries then a snapshot then more entries - and then read from the snapshot and the following entries" in { + "be able to record entries then a snapshot then more entries - and then read from the snapshot and the following entries - synchronous" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, false, null) @@ -155,7 +156,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA txlog.close } - "be able to record and delete entries" in { + "be able to record and delete entries - asynchronous" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, true, null) Thread.sleep(200) @@ -168,7 +169,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA Thread.sleep(200) intercept[BKNoSuchLedgerExistsException](TransactionLog.logFor(uuid, true, null)) } - "be able to record entries and read entries with 'entriesInRange'" in { + "be able to record entries and read entries with 'entriesInRange' - asynchronous" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, true, null) Thread.sleep(200) @@ -190,7 +191,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA txlog2.close } - "be able to record entries and read entries with 'entries'" in { + "be able to record entries and read entries with 'entries' - asynchronous" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, true, null) Thread.sleep(200) @@ -217,7 +218,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA txlog2.close } - "be able to record a snapshot" in { + "be able to record a snapshot - asynchronous" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, true, null) Thread.sleep(200) @@ -227,7 +228,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA txlog1.close } - "be able to record and read a snapshot and following entries" in { + "be able to record and read a snapshot and following entries - asynchronous" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, true, null) Thread.sleep(200) @@ -263,7 +264,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA txlog2.close } - "be able to record entries then a snapshot then more entries - and then read from the snapshot and the following entries" in { + "be able to record entries then a snapshot then more entries - and then read from the snapshot and the following entries - asynchronous" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, true, null) Thread.sleep(200) From b2bfe9a944f6d91227e41fc8b4e640b94531f68b Mon Sep 17 00:00:00 2001 From: Peter Vlugter Date: Thu, 4 Aug 2011 17:48:49 +1200 Subject: [PATCH 25/53] Some quietening of camel test output --- .../scala/akka/camel/TypedConsumerPublisher.scala | 4 ++-- .../main/scala/akka/camel/CamelContextLifecycle.scala | 6 +++--- akka-camel/src/test/resources/logback.xml | 11 +++++++++++ config/akka.test.conf | 1 + 4 files changed, 17 insertions(+), 5 deletions(-) create mode 100644 akka-camel/src/test/resources/logback.xml diff --git a/akka-camel-typed/src/main/scala/akka/camel/TypedConsumerPublisher.scala b/akka-camel-typed/src/main/scala/akka/camel/TypedConsumerPublisher.scala index e3b99e3535..a51f8ab42f 100644 --- a/akka-camel-typed/src/main/scala/akka/camel/TypedConsumerPublisher.scala +++ b/akka-camel-typed/src/main/scala/akka/camel/TypedConsumerPublisher.scala @@ -57,7 +57,7 @@ private[camel] object TypedConsumerPublisher { */ def handleConsumerMethodRegistered(event: ConsumerMethodRegistered) { CamelContextManager.mandatoryContext.addRoutes(new ConsumerMethodRouteBuilder(event)) - EventHandler notifyListeners EventHandler.Info(this, "published method %s of %s at endpoint %s" format (event.methodName, event.typedActor, event.endpointUri)) + EventHandler.info(this, "published method %s of %s at endpoint %s" format (event.methodName, event.typedActor, event.endpointUri)) } /** @@ -65,7 +65,7 @@ private[camel] object TypedConsumerPublisher { */ def handleConsumerMethodUnregistered(event: ConsumerMethodUnregistered) { CamelContextManager.mandatoryContext.stopRoute(event.methodUuid) - EventHandler notifyListeners EventHandler.Info(this, "unpublished method %s of %s from endpoint %s" format (event.methodName, event.typedActor, event.endpointUri)) + EventHandler.info(this, "unpublished method %s of %s from endpoint %s" format (event.methodName, event.typedActor, event.endpointUri)) } } diff --git a/akka-camel/src/main/scala/akka/camel/CamelContextLifecycle.scala b/akka-camel/src/main/scala/akka/camel/CamelContextLifecycle.scala index 9ff71e0f43..da472da927 100644 --- a/akka-camel/src/main/scala/akka/camel/CamelContextLifecycle.scala +++ b/akka-camel/src/main/scala/akka/camel/CamelContextLifecycle.scala @@ -102,7 +102,7 @@ trait CamelContextLifecycle { c.start t.start _started = true - EventHandler notifyListeners EventHandler.Info(this, "Camel context started") + EventHandler.info(this, "Camel context started") } } @@ -118,7 +118,7 @@ trait CamelContextLifecycle { c.stop _started = false _initialized = false - EventHandler notifyListeners EventHandler.Info(this, "Camel context stopped") + EventHandler.info(this, "Camel context stopped") } } @@ -141,7 +141,7 @@ trait CamelContextLifecycle { this._template = Some(context.createProducerTemplate) _initialized = true - EventHandler notifyListeners EventHandler.Info(this, "Camel context initialized") + EventHandler.info(this, "Camel context initialized") } } diff --git a/akka-camel/src/test/resources/logback.xml b/akka-camel/src/test/resources/logback.xml new file mode 100644 index 0000000000..023f2cd317 --- /dev/null +++ b/akka-camel/src/test/resources/logback.xml @@ -0,0 +1,11 @@ + + + + + [%4p] [%d{ISO8601}] [%t] %c{1}: %m%n + + + + + + diff --git a/config/akka.test.conf b/config/akka.test.conf index 025b5895fe..64583e4e98 100644 --- a/config/akka.test.conf +++ b/config/akka.test.conf @@ -6,4 +6,5 @@ include "akka-reference.conf" akka { event-handlers = ["akka.testkit.TestEventListener"] + event-handler-level = "WARNING" } From d5c0237b2f03abf9bc5bb7daeffa87fa1da7f2b9 Mon Sep 17 00:00:00 2001 From: Peter Vlugter Date: Thu, 4 Aug 2011 18:06:18 +1200 Subject: [PATCH 26/53] Disable parallel execution in global scope For avoiding tests from different projects, like cluster and zookeeper mailbox, to run in parallel --- project/AkkaBuild.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 84a16ef0cf..bd30d3b4fd 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -17,6 +17,7 @@ object AkkaBuild extends Build { id = "akka", base = file("."), settings = parentSettings ++ Unidoc.settings ++ rstdocSettings ++ Seq( + parallelExecution in GlobalScope := false, Unidoc.unidocExclude := Seq(samples.id, tutorials.id), rstdocDirectory <<= baseDirectory / "akka-docs" ), From 562ebd94901d8f04ae5d71becce92f5f99a9c38f Mon Sep 17 00:00:00 2001 From: Peter Vlugter Date: Fri, 5 Aug 2011 15:49:54 +1200 Subject: [PATCH 27/53] Quieten the stm test output --- .../example/UntypedCoordinatedCounter.java | 2 +- .../test/ExpectedFailureException.java | 7 ++++ .../test/UntypedCoordinatedCounter.java | 2 +- .../test/UntypedCoordinatedIncrementTest.java | 17 ++++++++ .../akka/transactor/test/UntypedCounter.java | 6 +-- .../akka/transactor/test/UntypedFailer.java | 2 +- .../test/UntypedTransactorTest.java | 17 ++++++++ .../transactor/CoordinatedIncrementSpec.scala | 38 ++++++++--------- .../scala/transactor/FickleFriendsSpec.scala | 41 ++++++++----------- .../scala/transactor/TransactorSpec.scala | 26 ++++++++---- 10 files changed, 99 insertions(+), 59 deletions(-) create mode 100644 akka-stm/src/test/java/akka/transactor/test/ExpectedFailureException.java diff --git a/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedCounter.java b/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedCounter.java index 6e78e8ea43..10420b4f9e 100644 --- a/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedCounter.java +++ b/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedCounter.java @@ -10,7 +10,7 @@ public class UntypedCoordinatedCounter extends UntypedActor { private Ref count = new Ref(0); private void increment() { - System.out.println("incrementing"); + //System.out.println("incrementing"); count.set(count.get() + 1); } diff --git a/akka-stm/src/test/java/akka/transactor/test/ExpectedFailureException.java b/akka-stm/src/test/java/akka/transactor/test/ExpectedFailureException.java new file mode 100644 index 0000000000..a4f1beb647 --- /dev/null +++ b/akka-stm/src/test/java/akka/transactor/test/ExpectedFailureException.java @@ -0,0 +1,7 @@ +package akka.transactor.test; + +public class ExpectedFailureException extends RuntimeException { + public ExpectedFailureException() { + super("Expected failure"); + } +} diff --git a/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedCounter.java b/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedCounter.java index 99f0cd19fe..abde886454 100644 --- a/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedCounter.java +++ b/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedCounter.java @@ -26,7 +26,7 @@ public class UntypedCoordinatedCounter extends UntypedActor { } private void increment() { - System.out.println(name + ": incrementing"); + //System.out.println(name + ": incrementing"); count.set(count.get() + 1); } diff --git a/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedIncrementTest.java b/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedIncrementTest.java index df969c2b27..28d725701d 100644 --- a/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedIncrementTest.java +++ b/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedIncrementTest.java @@ -10,13 +10,21 @@ import akka.actor.ActorRef; import akka.actor.UntypedActor; import akka.actor.UntypedActorFactory; import akka.dispatch.Future; +import akka.event.EventHandler; +import akka.testkit.EventFilter; +import akka.testkit.ErrorFilter; +import akka.testkit.TestEvent; +import akka.transactor.CoordinatedTransactionException; +import java.util.Arrays; import java.util.ArrayList; import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import scala.Option; +import scala.collection.JavaConverters; +import scala.collection.Seq; public class UntypedCoordinatedIncrementTest { List counters; @@ -63,6 +71,10 @@ public class UntypedCoordinatedIncrementTest { } @Test public void incrementNoCountersWithFailingTransaction() { + EventFilter expectedFailureFilter = (EventFilter) new ErrorFilter(ExpectedFailureException.class); + EventFilter coordinatedFilter = (EventFilter) new ErrorFilter(CoordinatedTransactionException.class); + Seq ignoreExceptions = seq(expectedFailureFilter, coordinatedFilter); + EventHandler.notify(new TestEvent.Mute(ignoreExceptions)); CountDownLatch incrementLatch = new CountDownLatch(numCounters); List actors = new ArrayList(counters); actors.add(failer); @@ -83,6 +95,11 @@ public class UntypedCoordinatedIncrementTest { } } } + EventHandler.notify(new TestEvent.UnMute(ignoreExceptions)); + } + + public Seq seq(A... args) { + return JavaConverters.collectionAsScalaIterableConverter(Arrays.asList(args)).asScala().toSeq(); } } diff --git a/akka-stm/src/test/java/akka/transactor/test/UntypedCounter.java b/akka-stm/src/test/java/akka/transactor/test/UntypedCounter.java index 65923e7f4b..14d6e453f9 100644 --- a/akka-stm/src/test/java/akka/transactor/test/UntypedCounter.java +++ b/akka-stm/src/test/java/akka/transactor/test/UntypedCounter.java @@ -28,7 +28,7 @@ public class UntypedCounter extends UntypedTransactor { } private void increment() { - System.out.println(name + ": incrementing"); + //System.out.println(name + ": incrementing"); count.set(count.get() + 1); } @@ -48,7 +48,7 @@ public class UntypedCounter extends UntypedTransactor { } @Override public void before(Object message) { - System.out.println(name + ": before transaction"); + //System.out.println(name + ": before transaction"); } public void atomically(Object message) { @@ -65,7 +65,7 @@ public class UntypedCounter extends UntypedTransactor { } @Override public void after(Object message) { - System.out.println(name + ": after transaction"); + //System.out.println(name + ": after transaction"); } @Override public boolean normally(Object message) { diff --git a/akka-stm/src/test/java/akka/transactor/test/UntypedFailer.java b/akka-stm/src/test/java/akka/transactor/test/UntypedFailer.java index 898dafa529..5e7328c566 100644 --- a/akka-stm/src/test/java/akka/transactor/test/UntypedFailer.java +++ b/akka-stm/src/test/java/akka/transactor/test/UntypedFailer.java @@ -4,6 +4,6 @@ import akka.transactor.UntypedTransactor; public class UntypedFailer extends UntypedTransactor { public void atomically(Object message) throws Exception { - throw new RuntimeException("Expected failure"); + throw new ExpectedFailureException(); } } diff --git a/akka-stm/src/test/java/akka/transactor/test/UntypedTransactorTest.java b/akka-stm/src/test/java/akka/transactor/test/UntypedTransactorTest.java index fb4cb5c0d5..f8ee344e5b 100644 --- a/akka-stm/src/test/java/akka/transactor/test/UntypedTransactorTest.java +++ b/akka-stm/src/test/java/akka/transactor/test/UntypedTransactorTest.java @@ -9,13 +9,21 @@ import akka.actor.Actors; import akka.actor.UntypedActor; import akka.actor.UntypedActorFactory; import akka.dispatch.Future; +import akka.event.EventHandler; +import akka.testkit.EventFilter; +import akka.testkit.ErrorFilter; +import akka.testkit.TestEvent; +import akka.transactor.CoordinatedTransactionException; +import java.util.Arrays; import java.util.ArrayList; import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import scala.Option; +import scala.collection.JavaConverters; +import scala.collection.Seq; public class UntypedTransactorTest { List counters; @@ -62,6 +70,10 @@ public class UntypedTransactorTest { } @Test public void incrementNoCountersWithFailingTransaction() { + EventFilter expectedFailureFilter = (EventFilter) new ErrorFilter(ExpectedFailureException.class); + EventFilter coordinatedFilter = (EventFilter) new ErrorFilter(CoordinatedTransactionException.class); + Seq ignoreExceptions = seq(expectedFailureFilter, coordinatedFilter); + EventHandler.notify(new TestEvent.Mute(ignoreExceptions)); CountDownLatch incrementLatch = new CountDownLatch(numCounters); List actors = new ArrayList(counters); actors.add(failer); @@ -82,6 +94,11 @@ public class UntypedTransactorTest { } } } + EventHandler.notify(new TestEvent.UnMute(ignoreExceptions)); + } + + public Seq seq(A... args) { + return JavaConverters.collectionAsScalaIterableConverter(Arrays.asList(args)).asScala().toSeq(); } } diff --git a/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala b/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala index 1683893b39..d05435b98b 100644 --- a/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala +++ b/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala @@ -8,9 +8,8 @@ import akka.transactor.Coordinated import akka.actor.{ Actor, ActorRef } import akka.stm.{ Ref, TransactionFactory } import akka.util.duration._ -import akka.event.EventHandler -import akka.testkit.EventFilter -import akka.testkit.TestEvent._ +import akka.transactor.CoordinatedTransactionException +import akka.testkit._ object CoordinatedIncrement { case class Increment(friends: Seq[ActorRef]) @@ -39,13 +38,15 @@ object CoordinatedIncrement { } } + class ExpectedFailureException extends RuntimeException("Expected failure") + class Failer extends Actor { val txFactory = TransactionFactory(timeout = 3 seconds) def receive = { case coordinated @ Coordinated(Increment(friends)) ⇒ { coordinated.atomic(txFactory) { - throw new RuntimeException("Expected failure") + throw new ExpectedFailureException } } } @@ -55,15 +56,6 @@ object CoordinatedIncrement { class CoordinatedIncrementSpec extends WordSpec with MustMatchers with BeforeAndAfterAll { import CoordinatedIncrement._ - override def beforeAll() { - EventHandler notify Mute(EventFilter[RuntimeException]("Expected failure")) - EventHandler notify Mute(EventFilter[org.multiverse.api.exceptions.DeadTransactionException]()) - } - - override def afterAll() { - EventHandler notify UnMuteAll - } - val numCounters = 5 val timeout = 5 seconds @@ -88,15 +80,19 @@ class CoordinatedIncrementSpec extends WordSpec with MustMatchers with BeforeAnd } "increment no counters with a failing transaction" in { - val (counters, failer) = createActors - val coordinated = Coordinated() - counters(0) ! Coordinated(Increment(counters.tail :+ failer)) - coordinated.await - for (counter ← counters) { - (counter ? GetCount).as[Int].get must be === 0 + filterException[ExpectedFailureException] { + filterException[CoordinatedTransactionException] { + val (counters, failer) = createActors + val coordinated = Coordinated() + counters(0) ! Coordinated(Increment(counters.tail :+ failer)) + coordinated.await + for (counter ← counters) { + (counter ? GetCount).as[Int].get must be === 0 + } + counters foreach (_.stop()) + failer.stop() + } } - counters foreach (_.stop()) - failer.stop() } } } diff --git a/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala b/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala index 78821f31c0..1f79afcdf6 100644 --- a/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala +++ b/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala @@ -8,9 +8,8 @@ import akka.transactor.Coordinated import akka.actor.{ Actor, ActorRef } import akka.stm._ import akka.util.duration._ -import akka.event.EventHandler -import akka.testkit.EventFilter -import akka.testkit.TestEvent._ +import akka.transactor.CoordinatedTransactionException +import akka.testkit._ import scala.util.Random.{ nextInt ⇒ random } @@ -59,6 +58,8 @@ object FickleFriends { } } + class ExpectedFailureException(message: String) extends RuntimeException(message) + /** * FickleCounter randomly fails at different points with 50% chance of failing overall. */ @@ -72,7 +73,7 @@ object FickleFriends { } def failIf(x: Int, y: Int) = { - if (x == y) throw new RuntimeException("Random fail at position " + x) + if (x == y) throw new ExpectedFailureException("Random fail at position " + x) } def receive = { @@ -98,16 +99,6 @@ object FickleFriends { class FickleFriendsSpec extends WordSpec with MustMatchers with BeforeAndAfterAll { import FickleFriends._ - val ignoreEvents = List(EventFilter(classOf[RuntimeException], message = "Random fail")) - - override def beforeAll() { - ignoreEvents foreach (f ⇒ EventHandler.notify(Mute(f))) - } - - override def afterAll() { - ignoreEvents foreach (f ⇒ EventHandler.notify(UnMute(f))) - } - val numCounters = 2 def createActors = { @@ -119,16 +110,20 @@ class FickleFriendsSpec extends WordSpec with MustMatchers with BeforeAndAfterAl "Coordinated fickle friends" should { "eventually succeed to increment all counters by one" in { - val (counters, coordinator) = createActors - val latch = new CountDownLatch(1) - coordinator ! FriendlyIncrement(counters, latch) - latch.await // this could take a while - (coordinator ? GetCount).as[Int].get must be === 1 - for (counter ← counters) { - (counter ? GetCount).as[Int].get must be === 1 + filterException[ExpectedFailureException] { + filterException[CoordinatedTransactionException] { + val (counters, coordinator) = createActors + val latch = new CountDownLatch(1) + coordinator ! FriendlyIncrement(counters, latch) + latch.await // this could take a while + (coordinator ? GetCount).as[Int].get must be === 1 + for (counter ← counters) { + (counter ? GetCount).as[Int].get must be === 1 + } + counters foreach (_.stop()) + coordinator.stop() + } } - counters foreach (_.stop()) - coordinator.stop() } } } diff --git a/akka-stm/src/test/scala/transactor/TransactorSpec.scala b/akka-stm/src/test/scala/transactor/TransactorSpec.scala index 8cccdf430c..2c72fd1650 100644 --- a/akka-stm/src/test/scala/transactor/TransactorSpec.scala +++ b/akka-stm/src/test/scala/transactor/TransactorSpec.scala @@ -7,6 +7,8 @@ import akka.transactor.Transactor import akka.actor.{ Actor, ActorRef } import akka.stm._ import akka.util.duration._ +import akka.transactor.CoordinatedTransactionException +import akka.testkit._ import java.util.concurrent.CountDownLatch @@ -51,9 +53,11 @@ object TransactorIncrement { } } + class ExpectedFailureException extends RuntimeException("Expected failure") + class Failer extends Transactor { def atomically = { - case _ ⇒ throw new RuntimeException("Expected failure") + case _ ⇒ throw new ExpectedFailureException } } } @@ -99,15 +103,19 @@ class TransactorSpec extends WordSpec with MustMatchers { } "increment no counters with a failing transaction" in { - val (counters, failer) = createTransactors - val failLatch = new CountDownLatch(numCounters + 1) - counters(0) ! Increment(counters.tail :+ failer, failLatch) - failLatch.await(timeout.length, timeout.unit) - for (counter ← counters) { - (counter ? GetCount).as[Int].get must be === 0 + filterException[ExpectedFailureException] { + filterException[CoordinatedTransactionException] { + val (counters, failer) = createTransactors + val failLatch = new CountDownLatch(numCounters + 1) + counters(0) ! Increment(counters.tail :+ failer, failLatch) + failLatch.await(timeout.length, timeout.unit) + for (counter ← counters) { + (counter ? GetCount).as[Int].get must be === 0 + } + counters foreach (_.stop()) + failer.stop() + } } - counters foreach (_.stop()) - failer.stop() } } From 0057acdd8107724e7bf84e46f86d4f0b8fa908f5 Mon Sep 17 00:00:00 2001 From: Peter Vlugter Date: Fri, 5 Aug 2011 18:41:29 +1200 Subject: [PATCH 28/53] Some more quietening of tests --- .../src/test/resources/log4j.properties | 58 +++++++++++++++++++ .../src/test/resources/logback.xml | 11 ++++ .../transactor/CoordinatedIncrementSpec.scala | 29 +++++----- .../scala/transactor/FickleFriendsSpec.scala | 31 +++++----- .../scala/transactor/TransactorSpec.scala | 29 +++++----- .../scala/akka/testkit/TestActorRefSpec.scala | 3 + 6 files changed, 121 insertions(+), 40 deletions(-) create mode 100644 akka-durable-mailboxes/akka-zookeeper-mailbox/src/test/resources/log4j.properties create mode 100644 akka-samples/akka-sample-camel/src/test/resources/logback.xml diff --git a/akka-durable-mailboxes/akka-zookeeper-mailbox/src/test/resources/log4j.properties b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/test/resources/log4j.properties new file mode 100644 index 0000000000..2d07c8e051 --- /dev/null +++ b/akka-durable-mailboxes/akka-zookeeper-mailbox/src/test/resources/log4j.properties @@ -0,0 +1,58 @@ +# Define some default values that can be overridden by system properties +zookeeper.root.logger=INFO, CONSOLE +zookeeper.console.threshold=OFF +zookeeper.log.dir=. +zookeeper.log.file=zookeeper.log +zookeeper.log.threshold=DEBUG +zookeeper.tracelog.dir=. +zookeeper.tracelog.file=zookeeper_trace.log + +# +# ZooKeeper Logging Configuration +# + +# Format is " (, )+ + +# DEFAULT: console appender only +log4j.rootLogger=${zookeeper.root.logger} + +# Example with rolling log file +#log4j.rootLogger=DEBUG, CONSOLE, ROLLINGFILE + +# Example with rolling log file and tracing +#log4j.rootLogger=TRACE, CONSOLE, ROLLINGFILE, TRACEFILE + +# +# Log INFO level and above messages to the console +# +log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +log4j.appender.CONSOLE.Threshold=${zookeeper.console.threshold} +log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout +log4j.appender.CONSOLE.layout.ConversionPattern=%d{ISO8601} [myid:%X{myid}] - %-5p [%t:%C{1}@%L] - %m%n + +# +# Add ROLLINGFILE to rootLogger to get log file output +# Log DEBUG level and above messages to a log file +log4j.appender.ROLLINGFILE=org.apache.log4j.RollingFileAppender +log4j.appender.ROLLINGFILE.Threshold=${zookeeper.log.threshold} +log4j.appender.ROLLINGFILE.File=${zookeeper.log.dir}/${zookeeper.log.file} + +# Max log file size of 10MB +log4j.appender.ROLLINGFILE.MaxFileSize=10MB +# uncomment the next line to limit number of backup files +#log4j.appender.ROLLINGFILE.MaxBackupIndex=10 + +log4j.appender.ROLLINGFILE.layout=org.apache.log4j.PatternLayout +log4j.appender.ROLLINGFILE.layout.ConversionPattern=%d{ISO8601} [myid:%X{myid}] - %-5p [%t:%C{1}@%L] - %m%n + + +# +# Add TRACEFILE to rootLogger to get log file output +# Log DEBUG level and above messages to a log file +log4j.appender.TRACEFILE=org.apache.log4j.FileAppender +log4j.appender.TRACEFILE.Threshold=TRACE +log4j.appender.TRACEFILE.File=${zookeeper.tracelog.dir}/${zookeeper.tracelog.file} + +log4j.appender.TRACEFILE.layout=org.apache.log4j.PatternLayout +### Notice we are including log4j's NDC here (%x) +log4j.appender.TRACEFILE.layout.ConversionPattern=%d{ISO8601} [myid:%X{myid}] - %-5p [%t:%C{1}@%L][%x] - %m%n diff --git a/akka-samples/akka-sample-camel/src/test/resources/logback.xml b/akka-samples/akka-sample-camel/src/test/resources/logback.xml new file mode 100644 index 0000000000..023f2cd317 --- /dev/null +++ b/akka-samples/akka-sample-camel/src/test/resources/logback.xml @@ -0,0 +1,11 @@ + + + + + [%4p] [%d{ISO8601}] [%t] %c{1}: %m%n + + + + + + diff --git a/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala b/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala index d05435b98b..c05f41624f 100644 --- a/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala +++ b/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala @@ -5,9 +5,10 @@ import org.scalatest.matchers.MustMatchers import org.scalatest.BeforeAndAfterAll import akka.transactor.Coordinated -import akka.actor.{ Actor, ActorRef } +import akka.actor.{ Actor, ActorRef, ActorTimeoutException } import akka.stm.{ Ref, TransactionFactory } import akka.util.duration._ +import akka.event.EventHandler import akka.transactor.CoordinatedTransactionException import akka.testkit._ @@ -80,19 +81,21 @@ class CoordinatedIncrementSpec extends WordSpec with MustMatchers with BeforeAnd } "increment no counters with a failing transaction" in { - filterException[ExpectedFailureException] { - filterException[CoordinatedTransactionException] { - val (counters, failer) = createActors - val coordinated = Coordinated() - counters(0) ! Coordinated(Increment(counters.tail :+ failer)) - coordinated.await - for (counter ← counters) { - (counter ? GetCount).as[Int].get must be === 0 - } - counters foreach (_.stop()) - failer.stop() - } + val ignoreExceptions = Seq( + EventFilter[ExpectedFailureException], + EventFilter[CoordinatedTransactionException], + EventFilter[ActorTimeoutException]) + EventHandler.notify(TestEvent.Mute(ignoreExceptions)) + val (counters, failer) = createActors + val coordinated = Coordinated() + counters(0) ! Coordinated(Increment(counters.tail :+ failer)) + coordinated.await + for (counter ← counters) { + (counter ? GetCount).as[Int].get must be === 0 } + counters foreach (_.stop()) + failer.stop() + EventHandler.notify(TestEvent.UnMute(ignoreExceptions)) } } } diff --git a/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala b/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala index 1f79afcdf6..89f8738c10 100644 --- a/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala +++ b/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala @@ -5,9 +5,10 @@ import org.scalatest.matchers.MustMatchers import org.scalatest.BeforeAndAfterAll import akka.transactor.Coordinated -import akka.actor.{ Actor, ActorRef } +import akka.actor.{ Actor, ActorRef, ActorTimeoutException } import akka.stm._ import akka.util.duration._ +import akka.event.EventHandler import akka.transactor.CoordinatedTransactionException import akka.testkit._ @@ -110,20 +111,22 @@ class FickleFriendsSpec extends WordSpec with MustMatchers with BeforeAndAfterAl "Coordinated fickle friends" should { "eventually succeed to increment all counters by one" in { - filterException[ExpectedFailureException] { - filterException[CoordinatedTransactionException] { - val (counters, coordinator) = createActors - val latch = new CountDownLatch(1) - coordinator ! FriendlyIncrement(counters, latch) - latch.await // this could take a while - (coordinator ? GetCount).as[Int].get must be === 1 - for (counter ← counters) { - (counter ? GetCount).as[Int].get must be === 1 - } - counters foreach (_.stop()) - coordinator.stop() - } + val ignoreExceptions = Seq( + EventFilter[ExpectedFailureException], + EventFilter[CoordinatedTransactionException], + EventFilter[ActorTimeoutException]) + EventHandler.notify(TestEvent.Mute(ignoreExceptions)) + val (counters, coordinator) = createActors + val latch = new CountDownLatch(1) + coordinator ! FriendlyIncrement(counters, latch) + latch.await // this could take a while + (coordinator ? GetCount).as[Int].get must be === 1 + for (counter ← counters) { + (counter ? GetCount).as[Int].get must be === 1 } + counters foreach (_.stop()) + coordinator.stop() + EventHandler.notify(TestEvent.UnMute(ignoreExceptions)) } } } diff --git a/akka-stm/src/test/scala/transactor/TransactorSpec.scala b/akka-stm/src/test/scala/transactor/TransactorSpec.scala index 2c72fd1650..86d3588814 100644 --- a/akka-stm/src/test/scala/transactor/TransactorSpec.scala +++ b/akka-stm/src/test/scala/transactor/TransactorSpec.scala @@ -4,9 +4,10 @@ import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers import akka.transactor.Transactor -import akka.actor.{ Actor, ActorRef } +import akka.actor.{ Actor, ActorRef, ActorTimeoutException } import akka.stm._ import akka.util.duration._ +import akka.event.EventHandler import akka.transactor.CoordinatedTransactionException import akka.testkit._ @@ -103,19 +104,21 @@ class TransactorSpec extends WordSpec with MustMatchers { } "increment no counters with a failing transaction" in { - filterException[ExpectedFailureException] { - filterException[CoordinatedTransactionException] { - val (counters, failer) = createTransactors - val failLatch = new CountDownLatch(numCounters + 1) - counters(0) ! Increment(counters.tail :+ failer, failLatch) - failLatch.await(timeout.length, timeout.unit) - for (counter ← counters) { - (counter ? GetCount).as[Int].get must be === 0 - } - counters foreach (_.stop()) - failer.stop() - } + val ignoreExceptions = Seq( + EventFilter[ExpectedFailureException], + EventFilter[CoordinatedTransactionException], + EventFilter[ActorTimeoutException]) + EventHandler.notify(TestEvent.Mute(ignoreExceptions)) + val (counters, failer) = createTransactors + val failLatch = new CountDownLatch(numCounters + 1) + counters(0) ! Increment(counters.tail :+ failer, failLatch) + failLatch.await(timeout.length, timeout.unit) + for (counter ← counters) { + (counter ? GetCount).as[Int].get must be === 0 } + counters foreach (_.stop()) + failer.stop() + EventHandler.notify(TestEvent.UnMute(ignoreExceptions)) } } diff --git a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala index 3e8ac34690..0fab0a54b6 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala @@ -229,6 +229,8 @@ class TestActorRefSpec extends WordSpec with MustMatchers with BeforeAndAfterEac val boss = Actor.actorOf(new Actor { def receive = { case _ ⇒ } }).start() val ref = TestActorRef[WorkerActor].start() + val filter = EventFilter.custom(_ ⇒ true) + EventHandler.notify(TestEvent.Mute(filter)) val log = TestActorRef[Logger] EventHandler.addListener(log) boss link ref @@ -236,6 +238,7 @@ class TestActorRefSpec extends WordSpec with MustMatchers with BeforeAndAfterEac la.count must be(1) la.msg must (include("supervisor") and include("CallingThreadDispatcher")) EventHandler.removeListener(log) + EventHandler.notify(TestEvent.UnMute(filter)) } "proxy apply for the underlying actor" in { From bb33a11ffd700346f94536aa242fd283ac723a86 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 5 Aug 2011 14:51:21 +0200 Subject: [PATCH 29/53] Adding parens to postStop in FSM, closing ticket #1079 --- akka-actor/src/main/scala/akka/actor/FSM.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/actor/FSM.scala b/akka-actor/src/main/scala/akka/actor/FSM.scala index b3e9c96e9e..78d03fe1e7 100644 --- a/akka-actor/src/main/scala/akka/actor/FSM.scala +++ b/akka-actor/src/main/scala/akka/actor/FSM.scala @@ -528,7 +528,7 @@ trait FSM[S, D] extends ListenerManagement { } } - override def postStop { terminate(stay withStopReason Shutdown) } + override def postStop() { terminate(stay withStopReason Shutdown) } private def terminate(nextState: State): Unit = { if (!currentState.stopReason.isDefined) { From c6bdd33d9e66eaa33f395eb9841b6e3e762d3774 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Fri, 5 Aug 2011 08:55:36 -0600 Subject: [PATCH 30/53] Future: make callback stack usable outside of DefaultPromise, make Future.flow use callback stack, hide java api from Scala --- .../test/scala/akka/actor/actor/IOActor.scala | 4 +- .../test/scala/akka/dispatch/FutureSpec.scala | 25 +++- .../akka/dispatch/PromiseStreamSpec.scala | 76 ++++++++--- .../src/main/scala/akka/dispatch/Future.scala | 126 +++++++----------- .../main/scala/akka/util/cps/package.scala | 24 ++-- 5 files changed, 141 insertions(+), 114 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/actor/IOActor.scala b/akka-actor-tests/src/test/scala/akka/actor/actor/IOActor.scala index 82e1734b45..faf36acf6b 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/actor/IOActor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/actor/IOActor.scala @@ -79,12 +79,12 @@ object IOActorSpec { val value = socket read length.toInt server.owner ? (('set, key, value)) map ((x: Any) ⇒ ByteString("+OK\r\n")) case Array("GET", key) ⇒ - server.owner ? (('get, key)) collect { + server.owner ? (('get, key)) map { case Some(b: ByteString) ⇒ ByteString("$" + b.length + "\r\n") ++ b case None ⇒ ByteString("$-1\r\n") } case Array("GETALL") ⇒ - server.owner ? 'getall collect { + server.owner ? 'getall map { case m: Map[_, _] ⇒ (ByteString("*" + (m.size * 2) + "\r\n") /: m) { case (result, (k: String, v: ByteString)) ⇒ diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala index ff5199e101..2f2317c5f7 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala @@ -146,7 +146,7 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd behave like futureWithResult { test ⇒ val actor1 = actorOf[TestActor].start() val actor2 = actorOf(new Actor { def receive = { case s: String ⇒ self reply s.toUpperCase } }).start() - val future = actor1 ? "Hello" flatMap { _ match { case s: String ⇒ actor2 ? s } } + val future = actor1 ? "Hello" flatMap { case s: String ⇒ actor2 ? s } future.await test(future, "WORLD") actor1.stop() @@ -158,7 +158,7 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd filterException[ArithmeticException] { val actor1 = actorOf[TestActor].start() val actor2 = actorOf(new Actor { def receive = { case s: String ⇒ self reply (s.length / 0) } }).start() - val future = actor1 ? "Hello" flatMap { _ match { case s: String ⇒ actor2 ? s } } + val future = actor1 ? "Hello" flatMap { case s: String ⇒ actor2 ? s } future.await test(future, "/ by zero") actor1.stop() @@ -166,6 +166,19 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd } } } + "will throw a MatchError when matching wrong type" must { + behave like futureWithException[MatchError] { test ⇒ + filterException[MatchError] { + val actor1 = actorOf[TestActor].start() + val actor2 = actorOf(new Actor { def receive = { case s: String ⇒ self reply s.toUpperCase } }).start() + val future = actor1 ? "Hello" flatMap { case i: Int ⇒ actor2 ? i } + future.await + test(future, "World (of class java.lang.String)") + actor1.stop() + actor2.stop() + } + } + } } "being tested" must { @@ -570,11 +583,15 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd flow { one << 1 } + one.await + assert(one.isCompleted) assert(List(two, simpleResult).forall(_.isCompleted == false)) flow { two << 9 } + two.await + assert(List(one, two).forall(_.isCompleted == true)) assert(simpleResult.get === 10) @@ -592,7 +609,7 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd lz.open() x1() + x2() } - assert(lx.isOpen) + assert(lx.tryAwaitUninterruptible(2000, TimeUnit.MILLISECONDS)) assert(!ly.isOpen) assert(!lz.isOpen) assert(List(x1, x2, y1, y2).forall(_.isCompleted == false)) @@ -723,7 +740,7 @@ class FutureSpec extends WordSpec with MustMatchers with Checkers with BeforeAnd import Future.flow import akka.util.cps._ - val count = 10000 + val count = 1000 val promises = List.fill(count)(Promise[Int]()) diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/PromiseStreamSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/PromiseStreamSpec.scala index 022da62c5b..6ed8686bcb 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/PromiseStreamSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/PromiseStreamSpec.scala @@ -3,6 +3,7 @@ package akka.dispatch import org.scalatest.junit.JUnitSuite import org.junit.Test import Future.flow +import akka.util.cps._ class PromiseStreamSpec extends JUnitSuite { @Test @@ -74,30 +75,36 @@ class PromiseStreamSpec extends JUnitSuite { @Test def pendingTest2 { val a, b, c, d = Promise[Int]() - val q = PromiseStream[Int]() + val q1, q2 = PromiseStream[Int]() val oneTwo = Future(List(1, 2)) - flow { a << q } flow { - b << q - q << 3 << 4 + a << q2 + b << q2 + q1 << 3 << 4 } - flow { c << q } flow { - q <<< oneTwo - d << q + q2 <<< oneTwo + c << q1 + d << q1 } - assert((a.get, b.get, c.get, d.get) === (1, 2, 3, 4)) + assert(a.get === 1) + assert(b.get === 2) + assert(c.get === 3) + assert(d.get === 4) } @Test def pendingEnqueueTest { - val a, b = Promise[Int]() val q = PromiseStream[Int]() - flow { a << q } - flow { b << q } - val c = q.dequeue() + val a = q.dequeue() + val b = q.dequeue() + val c, d = Promise[Int]() + flow { + c << q + d << q + } q ++= List(1, 2, 3, 4) - val d = q.dequeue() + assert(a.get === 1) assert(b.get === 2) assert(c.get === 3) @@ -113,9 +120,11 @@ class PromiseStreamSpec extends JUnitSuite { flow { a << qi b << qs + c << qi + } + flow { + qs << ("Hello", "World!", "Test") } - flow { qs << ("Hello", "World!", "Test") } - flow { c << qi } assert(a.get === 5) assert(b.get === "World!") assert(c.get === 4) @@ -123,11 +132,38 @@ class PromiseStreamSpec extends JUnitSuite { @Test def concurrentStressTest { - val q = PromiseStream[Int]() - Future((0 until 50000) foreach (v ⇒ Future(q enqueue v))) - val future = Future.sequence(List.fill(10)(Future(Future.sequence(List.fill(10000)(q.dequeue()))).flatMap(x ⇒ x))) map (_.flatten.sorted) - Future((50000 until 100000) foreach (v ⇒ Future(q enqueue v))) + val q = PromiseStream[Long]() + + flow { + var n = 0L + repeatC(50000) { + n += 1 + q << n + } + } + + val future = Future sequence { + List.fill(10) { + flow { + var total = 0L + repeatC(10000) { + val n = q() + total += n + } + total + } + } + } map (_.sum) + + flow { + var n = 50000L + repeatC(50000) { + n += 1 + q << n + } + } + val result = future.get - assert(result === List.range(0, 100000), "Result did not match 'List.range(0, 100000)'") + assert(result === (1L to 100000L).sum) } } diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 24ea405f4f..35b5cf216f 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -309,16 +309,40 @@ object Future { */ def flow[A](body: ⇒ A @cps[Future[Any]])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Future[A] = { val future = Promise[A](timeout) - //dispatcher dispatchTask { () ⇒ - (reify(body) foreachFull (future completeWithResult, future completeWithException): Future[Any]) onException { - case e: Exception ⇒ future completeWithException e - } - //} + dispatchTask({ () ⇒ + (reify(body) foreachFull (future completeWithResult, future completeWithException): Future[Any]) onException { + case e: Exception ⇒ future completeWithException e + } + }, true) future } + + private val _taskStack = new ThreadLocal[Option[Stack[() ⇒ Unit]]]() { + override def initialValue = None + } + + private[akka] def dispatchTask(task: () ⇒ Unit, force: Boolean = false)(implicit dispatcher: MessageDispatcher): Unit = + _taskStack.get match { + case Some(taskStack) if !force ⇒ taskStack push task + case _ ⇒ + dispatcher dispatchTask { () ⇒ + try { + val taskStack = Stack[() ⇒ Unit](task) + _taskStack set Some(taskStack) + while (taskStack.nonEmpty) { + val next = taskStack.pop() + try { + next.apply() + } catch { + case e ⇒ // FIXME + } + } + } finally { _taskStack set None } + } + } } -sealed trait Future[+T] { +sealed trait Future[+T] extends japi.Future[T] { implicit def dispatcher: MessageDispatcher @@ -327,12 +351,8 @@ sealed trait Future[+T] { * * Returns the result of this Future without blocking, by suspending execution and storing it as a * continuation until the result is available. - * - * If this Future is untyped (a Future[Nothing]), a type parameter must be explicitly provided or - * execution will fail. The normal result of getting a Future from an ActorRef using ? will return - * an untyped Future. */ - def apply[A >: T]()(implicit timeout: Timeout): A @cps[Future[Any]] = shift(this flatMap (_: A ⇒ Future[Any])) + def apply()(implicit timeout: Timeout): T @cps[Future[Any]] = shift(this flatMap (_: T ⇒ Future[Any])) /** * Blocks awaiting completion of this Future, then returns the resulting value, @@ -483,26 +503,8 @@ sealed trait Future[+T] { * } yield b + "-" + c * */ - final def collect[A](pf: PartialFunction[Any, A])(implicit timeout: Timeout): Future[A] = { - val future = new DefaultPromise[A](timeout) - onComplete { self ⇒ - future complete { - self.value.get match { - case Right(r) ⇒ - try { - if (pf isDefinedAt r) Right(pf(r)) - else Left(new MatchError(r)) - } catch { - case e: Exception ⇒ - EventHandler.error(e, this, e.getMessage) - Left(e) - } - case v ⇒ v.asInstanceOf[Either[Throwable, A]] - } - } - } - future - } + @deprecated("No longer needed, use 'map' instead. Removed in 2.0", "2.0") + final def collect[A](pf: PartialFunction[T, A])(implicit timeout: Timeout): Future[A] = this map pf /** * Creates a new Future that will handle any matching Throwable that this @@ -670,18 +672,21 @@ sealed trait Future[+T] { else r.right.toOption } else None } +} - /* Java API */ - final def onComplete[A >: T](proc: Procedure[Future[A]]): this.type = onComplete(proc(_)) +package japi { + /* Future Java API */ + trait Future[+T] { self: akka.dispatch.Future[T] ⇒ + private[japi] final def onComplete[A >: T](proc: Procedure[Future[A]]): this.type = self.onComplete(proc(_)) - final def map[A >: T, B](f: JFunc[A, B]): Future[B] = map(f(_)) + private[japi] final def map[A >: T, B](f: JFunc[A, B]): akka.dispatch.Future[B] = self.map(f(_)) - final def flatMap[A >: T, B](f: JFunc[A, Future[B]]): Future[B] = flatMap(f(_)) + private[japi] final def flatMap[A >: T, B](f: JFunc[A, akka.dispatch.Future[B]]): akka.dispatch.Future[B] = self.flatMap(f(_)) - final def foreach[A >: T](proc: Procedure[A]): Unit = foreach(proc(_)) - - final def filter(p: JFunc[Any, Boolean]): Future[Any] = filter(p(_)) + private[japi] final def foreach[A >: T](proc: Procedure[A]): Unit = self.foreach(proc(_)) + private[japi] final def filter(p: JFunc[Any, Boolean]): akka.dispatch.Future[Any] = self.filter(p(_)) + } } object Promise { @@ -700,10 +705,6 @@ object Promise { * Construct a completable channel */ def channel(timeout: Long = Actor.TIMEOUT)(implicit dispatcher: MessageDispatcher): ActorPromise = new ActorPromise(timeout) - - private[akka] val callbacksPendingExecution = new ThreadLocal[Option[Stack[() ⇒ Unit]]]() { - override def initialValue = None - } } /** @@ -838,7 +839,7 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi } def complete(value: Either[Throwable, T]): this.type = { - processCallbacks { + val callbacks = { _lock.lock try { if (_value.isEmpty) { //Only complete if we aren't expired @@ -858,38 +859,11 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi } } + if (callbacks.nonEmpty) Future.dispatchTask(() ⇒ callbacks foreach notifyCompleted) + this } - private def processCallbacks(callbacks: List[Future[T] ⇒ Unit]): Unit = { - if (callbacks.nonEmpty) { // Steps to ensure we don't run into a stack-overflow situation - @tailrec - def runCallbacks(rest: List[Future[T] ⇒ Unit], callbackStack: Stack[() ⇒ Unit]) { - if (rest.nonEmpty) { - notifyCompleted(rest.head) - while (callbackStack.nonEmpty) { callbackStack.pop().apply() } - runCallbacks(rest.tail, callbackStack) - } - } - - val pending = Promise.callbacksPendingExecution.get - if (pending.isDefined) { //Instead of nesting the calls to the callbacks (leading to stack overflow) - pending.get.push(() ⇒ { // Linearize/aggregate callbacks at top level and then execute - val doNotify = notifyCompleted _ //Hoist closure to avoid garbage - callbacks foreach doNotify - }) - } else { - dispatcher dispatchTask { () ⇒ - try { - val callbackStack = Stack[() ⇒ Unit]() // Allocate new aggregator for pending callbacks - Promise.callbacksPendingExecution.set(Some(callbackStack)) // Specify the callback aggregator - runCallbacks(callbacks, callbackStack) // Execute callbacks, if they trigger new callbacks, they are aggregated - } finally { Promise.callbacksPendingExecution.set(None) } // Ensure cleanup - } - } - } - } - def onComplete(func: Future[T] ⇒ Unit): this.type = { _lock.lock val notifyNow = try { @@ -903,7 +877,7 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi _lock.unlock } - if (notifyNow) processCallbacks(List(func)) + if (notifyNow) Future.dispatchTask(() ⇒ notifyCompleted(func)) this } @@ -927,7 +901,7 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi _lock.unlock } - if (runNow) func(this) + if (runNow) Future.dispatchTask(() ⇒ notifyCompleted(func)) } this @@ -937,7 +911,7 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi if (timeout.duration.isFinite) { value match { case Some(_) ⇒ this - case _ if isExpired ⇒ new KeptPromise[A](try { Right(fallback) } catch { case e: Exception ⇒ Left(e) }) + case _ if isExpired ⇒ Future[A](fallback) case _ ⇒ val promise = new DefaultPromise[A](Timeout.never) promise completeWith this @@ -1000,7 +974,7 @@ sealed class KeptPromise[T](suppliedValue: Either[Throwable, T])(implicit val di def complete(value: Either[Throwable, T]): this.type = this def onComplete(func: Future[T] ⇒ Unit): this.type = { - dispatcher dispatchTask (() ⇒ func(this)) //TODO: Use pending callback stack + Future dispatchTask (() ⇒ func(this)) this } def await(atMost: Duration): this.type = this diff --git a/akka-actor/src/main/scala/akka/util/cps/package.scala b/akka-actor/src/main/scala/akka/util/cps/package.scala index 29114f8407..7746c42280 100644 --- a/akka-actor/src/main/scala/akka/util/cps/package.scala +++ b/akka-actor/src/main/scala/akka/util/cps/package.scala @@ -5,13 +5,13 @@ import scala.util.continuations._ package object cps { def matchC[A, B, C, D](in: A)(pf: PartialFunction[A, B @cpsParam[C, D]]): B @cpsParam[C, D] = pf(in) - def loopC[A](block: ⇒ Unit @cps[A])(implicit loop: CPSLoop[A]): Unit @cps[A] = + def loopC[A, U](block: ⇒ U @cps[A])(implicit loop: CPSLoop[A]): Unit @cps[A] = loop.loopC(block) - def whileC[A](test: ⇒ Boolean)(block: ⇒ Unit @cps[A])(implicit loop: CPSLoop[A]): Unit @cps[A] = + def whileC[A, U](test: ⇒ Boolean)(block: ⇒ U @cps[A])(implicit loop: CPSLoop[A]): Unit @cps[A] = loop.whileC(test)(block) - def repeatC[A](times: Int)(block: ⇒ Unit @cps[A])(implicit loop: CPSLoop[A]): Unit @cps[A] = + def repeatC[A, U](times: Int)(block: ⇒ U @cps[A])(implicit loop: CPSLoop[A]): Unit @cps[A] = loop.repeatC(times)(block) } @@ -22,20 +22,20 @@ package cps { } trait CPSLoop[A] { - def loopC(block: ⇒ Unit @cps[A]): Unit @cps[A] - def whileC(test: ⇒ Boolean)(block: ⇒ Unit @cps[A]): Unit @cps[A] - def repeatC(times: Int)(block: ⇒ Unit @cps[A]): Unit @cps[A] + def loopC[U](block: ⇒ U @cps[A]): Unit @cps[A] + def whileC[U](test: ⇒ Boolean)(block: ⇒ U @cps[A]): Unit @cps[A] + def repeatC[U](times: Int)(block: ⇒ U @cps[A]): Unit @cps[A] } import akka.dispatch.{ Future, Promise } class FutureCPSLoop extends CPSLoop[Future[Any]] { - def loopC(block: ⇒ Unit @cps[Future[Any]]): Unit @cps[Future[Any]] = + def loopC[U](block: ⇒ U @cps[Future[Any]]): Unit @cps[Future[Any]] = shift { c: (Unit ⇒ Future[Any]) ⇒ Future(reify(block) flatMap (_ ⇒ reify(loopC(block))) foreach c) } - def whileC(test: ⇒ Boolean)(block: ⇒ Unit @cps[Future[Any]]): Unit @cps[Future[Any]] = + def whileC[U](test: ⇒ Boolean)(block: ⇒ U @cps[Future[Any]]): Unit @cps[Future[Any]] = shift { c: (Unit ⇒ Future[Any]) ⇒ if (test) Future(reify(block) flatMap (_ ⇒ reify(whileC(test)(block))) foreach c) @@ -43,7 +43,7 @@ package cps { Promise() completeWithResult (shiftUnitR[Unit, Future[Any]](()) foreach c) } - def repeatC(times: Int)(block: ⇒ Unit @cps[Future[Any]]): Unit @cps[Future[Any]] = + def repeatC[U](times: Int)(block: ⇒ U @cps[Future[Any]]): Unit @cps[Future[Any]] = shift { c: (Unit ⇒ Future[Any]) ⇒ if (times > 0) Future(reify(block) flatMap (_ ⇒ reify(repeatC(times - 1)(block))) foreach c) @@ -55,19 +55,19 @@ package cps { trait DefaultCPSLoop { implicit def defaultCPSLoop[A] = new CPSLoop[A] { - def loopC(block: ⇒ Unit @cps[A]): Unit @cps[A] = { + def loopC[U](block: ⇒ U @cps[A]): Unit @cps[A] = { block loopC(block) } - def whileC(test: ⇒ Boolean)(block: ⇒ Unit @cps[A]): Unit @cps[A] = { + def whileC[U](test: ⇒ Boolean)(block: ⇒ U @cps[A]): Unit @cps[A] = { if (test) { block whileC(test)(block) } } - def repeatC(times: Int)(block: ⇒ Unit @cps[A]): Unit @cps[A] = { + def repeatC[U](times: Int)(block: ⇒ U @cps[A]): Unit @cps[A] = { if (times > 0) { block repeatC(times - 1)(block) From b1c64652ce802e3760bb87104d266d6b94a61844 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Fri, 5 Aug 2011 20:02:31 -0600 Subject: [PATCH 31/53] IO: add method to retry current message --- akka-actor/src/main/scala/akka/actor/IO.scala | 34 ++++++++++++++++--- 1 file changed, 29 insertions(+), 5 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/IO.scala b/akka-actor/src/main/scala/akka/actor/IO.scala index 5f4be5bcde..03de91900a 100644 --- a/akka-actor/src/main/scala/akka/actor/IO.scala +++ b/akka-actor/src/main/scala/akka/actor/IO.scala @@ -130,9 +130,11 @@ object IO { } sealed trait IOSuspendable[+A] - private case class ByteStringLength(continuation: (ByteString) ⇒ IOSuspendable[Any], handle: Handle, message: MessageInvocation, length: Int) extends IOSuspendable[ByteString] - private case class ByteStringDelimited(continuation: (ByteString) ⇒ IOSuspendable[Any], handle: Handle, message: MessageInvocation, delimter: ByteString, inclusive: Boolean, scanned: Int) extends IOSuspendable[ByteString] - private case class ByteStringAny(continuation: (ByteString) ⇒ IOSuspendable[Any], handle: Handle, message: MessageInvocation) extends IOSuspendable[ByteString] + sealed trait CurrentMessage { def message: MessageInvocation } + private case class ByteStringLength(continuation: (ByteString) ⇒ IOSuspendable[Any], handle: Handle, message: MessageInvocation, length: Int) extends IOSuspendable[ByteString] with CurrentMessage + private case class ByteStringDelimited(continuation: (ByteString) ⇒ IOSuspendable[Any], handle: Handle, message: MessageInvocation, delimter: ByteString, inclusive: Boolean, scanned: Int) extends IOSuspendable[ByteString] with CurrentMessage + private case class ByteStringAny(continuation: (ByteString) ⇒ IOSuspendable[Any], handle: Handle, message: MessageInvocation) extends IOSuspendable[ByteString] with CurrentMessage + private case class Retry(message: MessageInvocation) extends IOSuspendable[Nothing] private case object Idle extends IOSuspendable[Nothing] } @@ -167,8 +169,11 @@ trait IO { case Connected(socket) ⇒ state(socket).connected = true run() - case Closed(handle) ⇒ + case msg @ Closed(handle) ⇒ _state -= handle // TODO: clean up better + if (_receiveIO.isDefinedAt(msg)) { + _next = reset { _receiveIO(msg); Idle } + } run() case msg if _next ne Idle ⇒ _messages enqueue self.currentMessage @@ -179,6 +184,14 @@ trait IO { def receiveIO: ReceiveIO + def retry(): Any @cps[IOSuspendable[Any]] = + shift { _: (Any ⇒ IOSuspendable[Any]) ⇒ + _next match { + case n: CurrentMessage ⇒ Retry(n.message) + case _ ⇒ Idle + } + } + private lazy val _receiveIO = receiveIO // only reinvoke messages from the original message to avoid stack overflow @@ -228,6 +241,10 @@ trait IO { _next = continuation(bytes) run() } + case Retry(message) ⇒ + message +=: _messages + _next = Idle + run() case Idle ⇒ reinvoke() } } @@ -355,10 +372,17 @@ private[akka] class IOWorker(ioManager: ActorRef, val bufferSize: Int) { case channel: ReadChannel ⇒ read(handle.asReadable, channel) } if (key.isWritable) key.channel match { - case channel: WriteChannel ⇒ write(handle.asWritable, channel) + case channel: WriteChannel ⇒ + try { + write(handle.asWritable, channel) + } catch { + case e: IOException ⇒ // ignore, let it fail on read to ensure + // nothing left in read buffer. + } } } catch { case e: CancelledKeyException ⇒ cleanup(handle) + case e: IOException ⇒ cleanup(handle) } } From 458724de348c37aa86f3da45e40aaf0f0844530c Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 5 Aug 2011 22:27:16 +0200 Subject: [PATCH 32/53] Reimplementing DefaultCompletableFuture to be as non-blocking internally as possible --- .../src/main/scala/akka/dispatch/Future.scala | 152 ++++++++---------- 1 file changed, 63 insertions(+), 89 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 35b5cf216f..295e70c121 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -21,7 +21,9 @@ import java.util.{ LinkedList ⇒ JLinkedList } import scala.annotation.tailrec import scala.collection.mutable.Stack import akka.util.{ Switch, Duration, BoxedType } -import java.util.concurrent.atomic.{ AtomicLong, AtomicBoolean } + +import java.util.concurrent.atomic.{ AtomicReference, AtomicBoolean } +import scala.Math class FutureTimeoutException(message: String, cause: Throwable = null) extends AkkaException(message, cause) @@ -771,6 +773,17 @@ trait Promise[T] extends Future[T] { } +//Companion object to FState, just to provide a cheap, immutable default entry +private[akka] object FState { + val empty = new FState[Nothing]() + def apply[T](): FState[T] = empty.asInstanceOf[FState[T]] +} + +/** + * Represents the internal state of the DefaultCompletableFuture + */ +private[akka] case class FState[T](value: Option[Either[Throwable, T]] = None, listeners: List[Future[T] ⇒ Unit] = Nil) + /** * The default concrete Future implementation. */ @@ -784,78 +797,53 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi def this(timeout: Long, timeunit: TimeUnit)(implicit dispatcher: MessageDispatcher) = this(Timeout(timeout, timeunit)) private val _startTimeInNanos = currentTimeInNanos - private val _lock = new ReentrantLock - private val _signal = _lock.newCondition - private var _value: Option[Either[Throwable, T]] = None - private var _listeners: List[Future[T] ⇒ Unit] = Nil + private val ref = new AtomicReference[FState[T]](FState()) - /** - * Must be called inside _lock.lock<->_lock.unlock - * Returns true if completed within the timeout - */ @tailrec private def awaitUnsafe(waitTimeNanos: Long): Boolean = { - if (_value.isEmpty && waitTimeNanos > 0) { - val start = currentTimeInNanos - val remainingNanos = try { - _signal.awaitNanos(waitTimeNanos) - } catch { - case e: InterruptedException ⇒ - waitTimeNanos - (currentTimeInNanos - start) - } - awaitUnsafe(remainingNanos) + if (value.isEmpty && waitTimeNanos > 0) { + val ms = NANOS.toMillis(waitTimeNanos) + val ns = (waitTimeNanos % 1000000l).toInt //As per object.wait spec + val start = System.nanoTime() + try { ref.synchronized { ref.wait(ms, ns) } } catch { case e: InterruptedException ⇒ } + + awaitUnsafe(waitTimeNanos - Math.abs(System.nanoTime() - start)) } else { - _value.isDefined + value.isDefined } } - def await(atMost: Duration) = { - _lock.lock() - try { - if (!atMost.isFinite && !timeout.duration.isFinite) { //If wait until infinity - while (_value.isEmpty) { _signal.await } - this - } else { //Limited wait - val time = if (!atMost.isFinite) timeLeft() //If atMost is infinity, use preset timeout - else if (!timeout.duration.isFinite) atMost.toNanos //If preset timeout is infinite, use atMost - else atMost.toNanos min timeLeft() //Otherwise use the smallest of them - if (awaitUnsafe(time)) this - else throw new FutureTimeoutException("Future timed out after [" + NANOS.toMillis(time) + "] ms") - } - } finally { _lock.unlock } - } + protected def awaitThenThrow(waitNanos: Long): this.type = + if (awaitUnsafe(waitNanos)) this + else throw new FutureTimeoutException("Futures timed out after [" + NANOS.toMillis(waitNanos) + "] milliseconds") - def await = await(timeout.duration) + def await(atMost: Duration) = awaitThenThrow(atMost.toNanos min timeLeft()) + + def await = awaitThenThrow(timeLeft()) def isExpired: Boolean = if (timeout.duration.isFinite) timeLeft() <= 0 else false - def value: Option[Either[Throwable, T]] = { - _lock.lock - try { - _value - } finally { - _lock.unlock - } - } + def value: Option[Either[Throwable, T]] = ref.get.value def complete(value: Either[Throwable, T]): this.type = { val callbacks = { - _lock.lock try { - if (_value.isEmpty) { //Only complete if we aren't expired - if (!isExpired) { - _value = Some(value) - val existingListeners = _listeners - _listeners = Nil - existingListeners - } else { - _listeners = Nil + @tailrec + def tryComplete: List[Future[T] ⇒ Unit] = { + val cur = ref.get + if (cur.value.isDefined) Nil + else if ( /*cur.value.isEmpty && */ isExpired) { + //Empty and expired, so remove listeners + ref.compareAndSet(cur, FState()) //Try to reset the state to the default, doesn't matter if it fails Nil + } else { + if (ref.compareAndSet(cur, FState(Option(value), Nil))) cur.listeners + else tryComplete } - } else Nil + } + tryComplete } finally { - _signal.signalAll - _lock.unlock + ref.synchronized { ref.notifyAll() } //Notify any evil blockers } } @@ -865,44 +853,34 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi } def onComplete(func: Future[T] ⇒ Unit): this.type = { - _lock.lock - val notifyNow = try { - if (_value.isEmpty) { - if (!isExpired) { //Only add the listener if the future isn't expired - _listeners ::= func - false - } else false //Will never run the callback since the future is expired - } else true - } finally { - _lock.unlock + @tailrec //Returns whether the future has already been completed or not + def tryAddCallback(): Boolean = { + val cur = ref.get + if (cur.value.isDefined) true + else if (isExpired) false + else if (ref.compareAndSet(cur, cur.copy(listeners = func :: cur.listeners))) false + else tryAddCallback() } + val notifyNow = tryAddCallback() + if (notifyNow) Future.dispatchTask(() ⇒ notifyCompleted(func)) this } def onTimeout(func: Future[T] ⇒ Unit): this.type = { - if (timeout.duration.isFinite) { - _lock.lock - val runNow = try { - if (_value.isEmpty) { - if (!isExpired) { - val runnable = new Runnable { - def run() { - if (!isCompleted) func(self) - } - } - Scheduler.scheduleOnce(runnable, timeLeft, NANOS) - false - } else true - } else false - } finally { - _lock.unlock - } + val runNow = + if (!timeout.duration.isFinite) false //Not possible + else if (value.isEmpty) { + if (!isExpired) { + val runnable = new Runnable { def run() { if (!isCompleted) func(DefaultPromise.this) } } + Scheduler.scheduleOnce(runnable, timeLeft, NANOS) + false + } else true + } else false - if (runNow) Future.dispatchTask(() ⇒ notifyCompleted(func)) - } + if (runNow) Future.dispatchTask(() ⇒ notifyCompleted(func)) this } @@ -926,11 +904,7 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi } else this private def notifyCompleted(func: Future[T] ⇒ Unit) { - try { - func(this) - } catch { - case e ⇒ EventHandler.error(e, this, "Future onComplete-callback raised an exception") - } + try { func(this) } catch { case e ⇒ EventHandler notify EventHandler.Error(e, this, "Future onComplete-callback raised an exception") } //TODO catch, everything? Really? } @inline From 9fb91e92c985dda9f16a9a65dc7e60ca804a1cbf Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sat, 6 Aug 2011 14:03:04 +0200 Subject: [PATCH 33/53] Removing awaitBlocking from Future since Futures cannot be completed after timed out, also cleaning up a lot of code to use pattern matching instead of if/else while simplifying and avoiding allocations --- .../src/main/scala/akka/dispatch/Future.scala | 223 +++++++++--------- 1 file changed, 109 insertions(+), 114 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 295e70c121..47e0432b11 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -206,7 +206,7 @@ object Futures { /** * Java API. - * Transforms a java.lang.Iterable[A] into a Future[java.lang.Iterable[B]] using the provided Function A => Future[B]. + * Transforms a java.lang.Iterable[A] into a Future[java.lang.Iterable[B]] using the provided Function A ⇒ Future[B]. * This is useful for performing a parallel map. For example, to apply a function to all items of a list * in parallel. */ @@ -221,15 +221,9 @@ object Futures { /** * Java API. - * Transforms a java.lang.Iterable[A] into a Future[java.lang.Iterable[B]] using the provided Function A => Future[B]. + * Transforms a java.lang.Iterable[A] into a Future[java.lang.Iterable[B]] using the provided Function A ⇒ Future[B]. * This is useful for performing a parallel map. For example, to apply a function to all items of a list * in parallel. - * - * def traverse[A, B, M[_] <: Traversable[_]](in: M[A], timeout: Long = Actor.TIMEOUT)(fn: A => Future[B])(implicit cbf: CanBuildFrom[M[A], B, M[B]]): Future[M[B]] = - * in.foldLeft(new DefaultPromise[Builder[B, M[B]]](timeout).completeWithResult(cbf(in)): Future[Builder[B, M[B]]]) { (fr, a) => - * val fb = fn(a.asInstanceOf[A]) - * for (r <- fr; b <-fb) yield (r += b) - * }.map(_.result) */ def traverse[A, B](in: JIterable[A], fn: JFunc[A, Future[B]]): Future[JIterable[B]] = traverse(in, Timeout.default, fn) } @@ -277,11 +271,11 @@ object Future { sequence(in)(cbf, timeout) /** - * Transforms a Traversable[A] into a Future[Traversable[B]] using the provided Function A => Future[B]. + * Transforms a Traversable[A] into a Future[Traversable[B]] using the provided Function A ⇒ Future[B]. * This is useful for performing a parallel map. For example, to apply a function to all items of a list * in parallel: *

-   * val myFutureList = Futures.traverse(myList)(x => Future(myFunc(x)))
+   * val myFutureList = Futures.traverse(myList)(x ⇒ Future(myFunc(x)))
    * 
*/ def traverse[A, B, M[_] <: Traversable[_]](in: M[A])(fn: A ⇒ Future[B])(implicit cbf: CanBuildFrom[M[A], B, M[B]], timeout: Timeout): Future[M[B]] = @@ -384,20 +378,20 @@ sealed trait Future[+T] extends japi.Future[T] { /** * Await completion of this Future (as `await`) and return its value if it * conforms to A's erased type. - * - * def as[A](implicit m: Manifest[A]): Option[A] = - * try { - * await - * value match { - * case None ⇒ None - * case Some(_: Left[_, _]) ⇒ None - * case Some(Right(v)) ⇒ Some(BoxedType(m.erasure).cast(v).asInstanceOf[A]) - * } - * } catch { - * case _: Exception ⇒ None - * } */ + def as[A](implicit m: Manifest[A]): Option[A] = + try { + await + value match { + case None ⇒ None + case Some(_: Left[_, _]) ⇒ None + case Some(Right(v)) ⇒ Some(BoxedType(m.erasure).cast(v).asInstanceOf[A]) + } + } catch { + case _: Exception ⇒ None + } + /** * Tests whether this Future has been completed. */ @@ -429,19 +423,17 @@ sealed trait Future[+T] extends japi.Future[T] { /** * Returns the successful result of this Future if it exists. */ - final def result: Option[T] = { - val v = value - if (v.isDefined) v.get.right.toOption - else None + final def result: Option[T] = value match { + case Some(r) ⇒ r.right.toOption + case _ ⇒ None } /** * Returns the contained exception of this Future if it exists. */ - final def exception: Option[Throwable] = { - val v = value - if (v.isDefined) v.get.left.toOption - else None + final def exception: Option[Throwable] = value match { + case Some(r) ⇒ r.left.toOption + case _ ⇒ None } /** @@ -455,17 +447,29 @@ sealed trait Future[+T] extends japi.Future[T] { * When the future is completed with a valid result, apply the provided * PartialFunction to the result. *
-   *   val result = future onResult {
-   *     case Foo => "foo"
-   *     case Bar => "bar"
+   *   future receive {
+   *     case Foo ⇒ target ! "foo"
+   *     case Bar ⇒ target ! "bar"
    *   }
    * 
*/ - final def onResult(pf: PartialFunction[T, Unit]): this.type = onComplete { f ⇒ - val optr = f.result - if (optr.isDefined) { - val r = optr.get - if (pf isDefinedAt r) pf(r) + @deprecated("Use `onResult` instead, will be removed in the future", "1.2") + final def receive(pf: PartialFunction[Any, Unit]): this.type = onResult(pf) + + /** + * When the future is completed with a valid result, apply the provided + * PartialFunction to the result. + *
+   *   future onResult {
+   *     case Foo ⇒ target ! "foo"
+   *     case Bar ⇒ target ! "bar"
+   *   }
+   * 
+ */ + final def onResult(pf: PartialFunction[T, Unit]): this.type = onComplete { + _.value match { + case Some(Right(r)) if pf isDefinedAt r ⇒ pf(r) + case _ ⇒ } } @@ -473,17 +477,15 @@ sealed trait Future[+T] extends japi.Future[T] { * When the future is completed with an exception, apply the provided * PartialFunction to the exception. *
-   *   val result = future onException {
-   *     case Foo => "foo"
-   *     case Bar => "bar"
+   *   future onException {
+   *     case NumberFormatException ⇒ target ! "wrong format"
    *   }
    * 
*/ - final def onException(pf: PartialFunction[Throwable, Unit]): Future[T] = onComplete { f ⇒ - val opte = f.exception - if (opte.isDefined) { - val e = opte.get - if (pf isDefinedAt e) pf(e) + final def onException(pf: PartialFunction[Throwable, Unit]): this.type = onComplete { + _.value match { + case Some(Left(ex)) if pf isDefinedAt ex ⇒ pf(ex) + case _ ⇒ } } @@ -514,26 +516,31 @@ sealed trait Future[+T] extends japi.Future[T] { * a valid result then the new Future will contain the same. * Example: *
-   * Future(6 / 0) recover { case e: ArithmeticException => 0 } // result: 0
-   * Future(6 / 0) recover { case e: NotFoundException   => 0 } // result: exception
-   * Future(6 / 2) recover { case e: ArithmeticException => 0 } // result: 3
+   * Future(6 / 0) failure { case e: ArithmeticException ⇒ 0 } // result: 0
+   * Future(6 / 0) failure { case e: NotFoundException   ⇒ 0 } // result: exception
+   * Future(6 / 2) failure { case e: ArithmeticException ⇒ 0 } // result: 3
+   * 
+ */ + @deprecated("will be replaced by `recover`", "1.2") + final def failure[A >: T](pf: PartialFunction[Throwable, A]): Future[A] = recover(pf) + + /** + * Creates a new Future that will handle any matching Throwable that this + * Future might contain. If there is no match, or if this Future contains + * a valid result then the new Future will contain the same. + * Example: + *
+   * Future(6 / 0) failure { case e: ArithmeticException ⇒ 0 } // result: 0
+   * Future(6 / 0) failure { case e: NotFoundException   ⇒ 0 } // result: exception
+   * Future(6 / 2) failure { case e: ArithmeticException ⇒ 0 } // result: 3
    * 
*/ final def recover[A >: T](pf: PartialFunction[Throwable, A])(implicit timeout: Timeout): Future[A] = { val future = new DefaultPromise[A](timeout) - onComplete { self ⇒ - future complete { - self.value.get match { - case Left(e) ⇒ - try { - if (pf isDefinedAt e) Right(pf(e)) - else Left(e) - } catch { - case x: Exception ⇒ - Left(x) - } - case v ⇒ v - } + onComplete { + _.value.get match { + case Left(e) if pf isDefinedAt e ⇒ future.complete(try { Right(pf(e)) } catch { case x: Exception ⇒ Left(x) }) + case otherwise ⇒ future complete otherwise } } future @@ -554,19 +561,17 @@ sealed trait Future[+T] extends japi.Future[T] { */ final def map[A](f: T ⇒ A)(implicit timeout: Timeout): Future[A] = { val future = new DefaultPromise[A](timeout) - onComplete { self ⇒ - future complete { - self.value.get match { - case Right(r) ⇒ - try { - Right(f(r)) - } catch { - case e: Exception ⇒ - EventHandler.error(e, this, e.getMessage) - Left(e) - } - case v ⇒ v.asInstanceOf[Either[Throwable, A]] - } + onComplete { + _.value.get match { + case l: Left[_, _] ⇒ future complete l.asInstanceOf[Either[Throwable, A]] + case Right(res) ⇒ + future complete (try { + Right(f(res)) + } catch { + case e: Exception ⇒ + EventHandler.error(e, this, e.getMessage) + Left(e) + }) } } future @@ -608,26 +613,26 @@ sealed trait Future[+T] extends japi.Future[T] { */ final def flatMap[A](f: T ⇒ Future[A])(implicit timeout: Timeout): Future[A] = { val future = new DefaultPromise[A](timeout) + onComplete { _.value.get match { - case Right(r) ⇒ - try { - future completeWith f(r) - } catch { - case e: Exception ⇒ - EventHandler.error(e, this, e.getMessage) - future complete Left(e) - } - case v ⇒ future complete v.asInstanceOf[Either[Throwable, A]] + case l: Left[_, _] ⇒ future complete l.asInstanceOf[Either[Throwable, A]] + case Right(r) ⇒ try { + future.completeWith(f(r)) + } catch { + case e: Exception ⇒ + EventHandler.error(e, this, e.getMessage) + future complete Left(e) + } } } future } final def foreach(f: T ⇒ Unit): Unit = onComplete { - _.result match { - case Some(v) ⇒ f(v) - case None ⇒ + _.value.get match { + case Right(r) ⇒ f(r) + case _ ⇒ } } @@ -642,22 +647,16 @@ sealed trait Future[+T] extends japi.Future[T] { final def filter(p: T ⇒ Boolean)(implicit timeout: Timeout): Future[T] = { val future = new DefaultPromise[T](timeout) - onComplete { self ⇒ - future complete { - self.value.get match { - case Right(r) ⇒ - try { - if (p(r)) - Right(r) - else - Left(new MatchError(r)) - } catch { - case e: Exception ⇒ - EventHandler.error(e, this, e.getMessage) - Left(e) - } - case v ⇒ v - } + onComplete { + _.value.get match { + case l: Left[_, _] ⇒ future complete l.asInstanceOf[Either[Throwable, T]] + case r @ Right(res) ⇒ future complete (try { + if (p(res)) r else Left(new MatchError(res)) + } catch { + case e: Exception ⇒ + EventHandler.error(e, this, e.getMessage) + Left(e) + }) } } future @@ -666,13 +665,10 @@ sealed trait Future[+T] extends japi.Future[T] { /** * Returns the current result, throws the exception if one has been raised, else returns None */ - final def resultOrException: Option[T] = { - val v = value - if (v.isDefined) { - val r = v.get - if (r.isLeft) throw r.left.get - else r.right.toOption - } else None + final def resultOrException: Option[T] = value match { + case Some(Left(e)) ⇒ throw e + case Some(Right(r)) ⇒ Some(r) + case _ ⇒ None } } @@ -834,6 +830,7 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi if (cur.value.isDefined) Nil else if ( /*cur.value.isEmpty && */ isExpired) { //Empty and expired, so remove listeners + //TODO Perhaps cancel existing onTimeout listeners in the future here? ref.compareAndSet(cur, FState()) //Try to reset the state to the default, doesn't matter if it fails Nil } else { @@ -862,9 +859,7 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi else tryAddCallback() } - val notifyNow = tryAddCallback() - - if (notifyNow) Future.dispatchTask(() ⇒ notifyCompleted(func)) + if (tryAddCallback()) Future.dispatchTask(() ⇒ notifyCompleted(func)) this } @@ -874,7 +869,7 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi if (!timeout.duration.isFinite) false //Not possible else if (value.isEmpty) { if (!isExpired) { - val runnable = new Runnable { def run() { if (!isCompleted) func(DefaultPromise.this) } } + val runnable = new Runnable { def run() { if (!isCompleted) func(DefaultPromise.this) } } //TODO Reschedule is run prematurely Scheduler.scheduleOnce(runnable, timeLeft, NANOS) false } else true From 811e14e081aed754f83109c537453dceb4a8dd0b Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sat, 6 Aug 2011 20:05:43 +0200 Subject: [PATCH 34/53] Fixing await so that it respects infinite timeouts --- .../src/main/scala/akka/dispatch/Future.scala | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 47e0432b11..d01085e4df 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -809,13 +809,21 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi } } - protected def awaitThenThrow(waitNanos: Long): this.type = + def await(atMost: Duration): this.type = { + val waitNanos = + if (timeout.duration.isFinite && atMost.isFinite) + atMost.toNanos min timeLeft() + else if (atMost.isFinite) + atMost.toNanos + else if (timeout.duration.isFinite) + timeLeft() + else Long.MaxValue //If both are infinite, use Long.MaxValue + if (awaitUnsafe(waitNanos)) this else throw new FutureTimeoutException("Futures timed out after [" + NANOS.toMillis(waitNanos) + "] milliseconds") + } - def await(atMost: Duration) = awaitThenThrow(atMost.toNanos min timeLeft()) - - def await = awaitThenThrow(timeLeft()) + def await = await(timeout.duration) def isExpired: Boolean = if (timeout.duration.isFinite) timeLeft() <= 0 else false From 8a1d31691c573862d24d6677858ca3910b68c2a9 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sat, 6 Aug 2011 20:44:49 +0200 Subject: [PATCH 35/53] Removing deprecated methods from Future and removing one of the bad guys _as_ --- .../src/main/scala/akka/dispatch/Future.scala | 69 ++----------------- 1 file changed, 4 insertions(+), 65 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index d01085e4df..3b8be4bb94 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -375,23 +375,6 @@ sealed trait Future[+T] extends japi.Future[T] { */ def await(atMost: Duration): Future[T] - /** - * Await completion of this Future (as `await`) and return its value if it - * conforms to A's erased type. - */ - - def as[A](implicit m: Manifest[A]): Option[A] = - try { - await - value match { - case None ⇒ None - case Some(_: Left[_, _]) ⇒ None - case Some(Right(v)) ⇒ Some(BoxedType(m.erasure).cast(v).asInstanceOf[A]) - } - } catch { - case _: Exception ⇒ None - } - /** * Tests whether this Future has been completed. */ @@ -443,19 +426,6 @@ sealed trait Future[+T] extends japi.Future[T] { */ def onComplete(func: Future[T] ⇒ Unit): this.type - /** - * When the future is completed with a valid result, apply the provided - * PartialFunction to the result. - *
-   *   future receive {
-   *     case Foo ⇒ target ! "foo"
-   *     case Bar ⇒ target ! "bar"
-   *   }
-   * 
- */ - @deprecated("Use `onResult` instead, will be removed in the future", "1.2") - final def receive(pf: PartialFunction[Any, Unit]): this.type = onResult(pf) - /** * When the future is completed with a valid result, apply the provided * PartialFunction to the result. @@ -493,37 +463,6 @@ sealed trait Future[+T] extends japi.Future[T] { def orElse[A >: T](fallback: ⇒ A): Future[A] - /** - * Creates a new Future by applying a PartialFunction to the successful - * result of this Future if a match is found, or else return a MatchError. - * If this Future is completed with an exception then the new Future will - * also contain this exception. - * Example: - *
-   * val future1 = for {
-   *   a <- actor ? Req("Hello") collect { case Res(x: Int)    => x }
-   *   b <- actor ? Req(a)       collect { case Res(x: String) => x }
-   *   c <- actor ? Req(7)       collect { case Res(x: String) => x }
-   * } yield b + "-" + c
-   * 
- */ - @deprecated("No longer needed, use 'map' instead. Removed in 2.0", "2.0") - final def collect[A](pf: PartialFunction[T, A])(implicit timeout: Timeout): Future[A] = this map pf - - /** - * Creates a new Future that will handle any matching Throwable that this - * Future might contain. If there is no match, or if this Future contains - * a valid result then the new Future will contain the same. - * Example: - *
-   * Future(6 / 0) failure { case e: ArithmeticException ⇒ 0 } // result: 0
-   * Future(6 / 0) failure { case e: NotFoundException   ⇒ 0 } // result: exception
-   * Future(6 / 2) failure { case e: ArithmeticException ⇒ 0 } // result: 3
-   * 
- */ - @deprecated("will be replaced by `recover`", "1.2") - final def failure[A >: T](pf: PartialFunction[Throwable, A]): Future[A] = recover(pf) - /** * Creates a new Future that will handle any matching Throwable that this * Future might contain. If there is no match, or if this Future contains @@ -800,10 +739,10 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi if (value.isEmpty && waitTimeNanos > 0) { val ms = NANOS.toMillis(waitTimeNanos) val ns = (waitTimeNanos % 1000000l).toInt //As per object.wait spec - val start = System.nanoTime() + val start = currentTimeInNanos try { ref.synchronized { ref.wait(ms, ns) } } catch { case e: InterruptedException ⇒ } - awaitUnsafe(waitTimeNanos - Math.abs(System.nanoTime() - start)) + awaitUnsafe(waitTimeNanos - (currentTimeInNanos - start)) } else { value.isDefined } @@ -894,7 +833,7 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi case Some(_) ⇒ this case _ if isExpired ⇒ Future[A](fallback) case _ ⇒ - val promise = new DefaultPromise[A](Timeout.never) + val promise = new DefaultPromise[A](Timeout.never) //TODO FIXME We can't have infinite timeout here, doesn't make sense. promise completeWith this val runnable = new Runnable { def run() { @@ -911,7 +850,7 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi } @inline - private def currentTimeInNanos: Long = MILLIS.toNanos(System.currentTimeMillis) + private def currentTimeInNanos: Long = MILLIS.toNanos(System.currentTimeMillis) //TODO Switch to math.abs(System.nanoTime)? @inline private def timeLeft(): Long = timeoutInNanos - (currentTimeInNanos - _startTimeInNanos) } From bc1f7565b7ff175c4bf6e9c08a8d30f9c1983cc8 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Sat, 6 Aug 2011 13:32:03 -0600 Subject: [PATCH 36/53] Fixed race in Future.await, and minor changes to Future.result and Future.exception --- akka-actor/src/main/scala/akka/dispatch/Future.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 3b8be4bb94..cfd729a25e 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -407,16 +407,16 @@ sealed trait Future[+T] extends japi.Future[T] { * Returns the successful result of this Future if it exists. */ final def result: Option[T] = value match { - case Some(r) ⇒ r.right.toOption - case _ ⇒ None + case Some(Right(r)) ⇒ Some(r) + case _ ⇒ None } /** * Returns the contained exception of this Future if it exists. */ final def exception: Option[Throwable] = value match { - case Some(r) ⇒ r.left.toOption - case _ ⇒ None + case Some(Left(e)) ⇒ Some(e) + case _ ⇒ None } /** @@ -740,7 +740,7 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi val ms = NANOS.toMillis(waitTimeNanos) val ns = (waitTimeNanos % 1000000l).toInt //As per object.wait spec val start = currentTimeInNanos - try { ref.synchronized { ref.wait(ms, ns) } } catch { case e: InterruptedException ⇒ } + try { ref.synchronized { if (value.isEmpty) ref.wait(ms, ns) } } catch { case e: InterruptedException ⇒ } awaitUnsafe(waitTimeNanos - (currentTimeInNanos - start)) } else { From aaec3aef77d7780a006bcac0ce0d6de2db77e7b7 Mon Sep 17 00:00:00 2001 From: Peter Veentjer Date: Sat, 6 Aug 2011 23:08:01 +0300 Subject: [PATCH 37/53] ticket #992 --- .../src/main/scala/akka/cluster/Cluster.scala | 368 ++++++++++-------- .../scala/akka/cluster/TransactionLog.scala | 169 +++++--- .../scala/akka/cluster/ClusterTestNode.scala | 22 -- .../akka/cluster/MasterClusterTestNode.scala | 31 ++ ...LogWriteBehindNoSnapshotMultiJvmSpec.scala | 9 +- ...onLogWriteBehindSnapshotMultiJvmSpec.scala | 9 +- ...LogWriteBehindNoSnapshotMultiJvmNode1.conf | 2 - ...LogWriteBehindNoSnapshotMultiJvmNode2.conf | 1 - ...ogWriteThroughNoSnapshotMultiJvmNode1.conf | 1 - ...ogWriteThroughNoSnapshotMultiJvmNode2.conf | 1 - ...ogWriteThroughNoSnapshotMultiJvmSpec.scala | 11 +- ...nLogWriteThroughSnapshotMultiJvmSpec.scala | 9 +- .../akka/cluster/TransactionLogSpec.scala | 108 ++++- 13 files changed, 439 insertions(+), 302 deletions(-) create mode 100644 akka-cluster/src/multi-jvm/scala/akka/cluster/MasterClusterTestNode.scala diff --git a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala index 348b11195e..2b8a14a41b 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala @@ -1758,6 +1758,8 @@ object RemoteClusterDaemon { /** * Internal "daemon" actor for cluster internal communication. * + * It acts as the brain of the cluster that responds to cluster events (messages) and undertakes action. + * * @author
Jonas Bonér */ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor { @@ -1774,185 +1776,207 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor { def receive: Receive = { case message: RemoteDaemonMessageProtocol ⇒ EventHandler.debug(this, - "Received command [\n%s] to RemoteClusterDaemon on node [%s]" - .format(message, cluster.nodeAddress.nodeName)) + "Received command [\n%s] to RemoteClusterDaemon on node [%s]".format(message, cluster.nodeAddress.nodeName)) message.getMessageType match { - - case USE ⇒ - try { - if (message.hasActorAddress) { - val actorAddress = message.getActorAddress - cluster.serializerForActor(actorAddress) foreach { serializer ⇒ - cluster.use(actorAddress, serializer) foreach { newActorRef ⇒ - cluster.remoteService.register(actorAddress, newActorRef) - - if (message.hasReplicateActorFromUuid) { - // replication is used - fetch the messages and replay them - import akka.remote.protocol.RemoteProtocol._ - import akka.remote.MessageSerializer - - val replicateFromUuid = uuidProtocolToUuid(message.getReplicateActorFromUuid) - val deployment = Deployer.deploymentFor(actorAddress) - val replicationScheme = DeploymentConfig.replicationSchemeFor(deployment).getOrElse( - throw new IllegalStateException( - "Actor [" + actorAddress + "] should have been configured as a replicated actor but could not find its ReplicationScheme")) - val isWriteBehind = DeploymentConfig.isWriteBehindReplication(replicationScheme) - - try { - // get the transaction log for the actor UUID - val txLog = TransactionLog.logFor(replicateFromUuid.toString, isWriteBehind, replicationScheme) - - // get the latest snapshot (Option[Array[Byte]]) and all the subsequent messages (Array[Byte]) - val (snapshotAsBytes, entriesAsBytes) = txLog.latestSnapshotAndSubsequentEntries - - // deserialize and restore actor snapshot - val actorRefToUseForReplay = - snapshotAsBytes match { - - // we have a new actor ref - the snapshot - case Some(bytes) ⇒ - // stop the new actor ref and use the snapshot instead - cluster.remoteService.unregister(actorAddress) - - // deserialize the snapshot actor ref and register it as remote actor - val uncompressedBytes = - if (Cluster.shouldCompressData) LZF.uncompress(bytes) - else bytes - - val snapshotActorRef = fromBinary(uncompressedBytes, newActorRef.uuid).start() - cluster.remoteService.register(actorAddress, snapshotActorRef) - - // FIXME we should call 'stop()' here (to GC the actor), but can't since that will currently shut down the TransactionLog for this UUID - since both this actor and the new snapshotActorRef have the same UUID (which they should) - //newActorRef.stop() - - snapshotActorRef - - // we have no snapshot - use the new actor ref - case None ⇒ - newActorRef - } - - // deserialize the messages - val messages: Vector[AnyRef] = entriesAsBytes map { bytes ⇒ - val messageBytes = - if (Cluster.shouldCompressData) LZF.uncompress(bytes) - else bytes - MessageSerializer.deserialize(MessageProtocol.parseFrom(messageBytes), None) - } - - EventHandler.info(this, "Replaying [%s] messages to actor [%s]".format(messages.size, actorAddress)) - - // replay all messages - messages foreach { message ⇒ - EventHandler.debug(this, "Replaying message [%s] to actor [%s]".format(message, actorAddress)) - - // FIXME how to handle '?' messages? We can *not* replay them with the correct semantics. Should we: 1. Ignore/drop them and log warning? 2. Throw exception when about to log them? 3. Other? - actorRefToUseForReplay ! message - } - - } catch { - case e: Throwable ⇒ - EventHandler.error(e, this, e.toString) - throw e - } - } - } - } - } else { - EventHandler.error(this, "Actor 'address' is not defined, ignoring remote cluster daemon command [%s]".format(message)) - } - self.reply(Success) - - } catch { - case error ⇒ - self.reply(Failure(error)) - throw error - } - - case RELEASE ⇒ - if (message.hasActorUuid) { - cluster.actorAddressForUuid(uuidProtocolToUuid(message.getActorUuid)) foreach { address ⇒ - cluster.release(address) - } - } else if (message.hasActorAddress) { - cluster release message.getActorAddress - } else { - EventHandler.warning(this, - "None of 'uuid' or 'actorAddress'' is specified, ignoring remote cluster daemon command [%s]" - .format(message)) - } - - case START ⇒ cluster.start() - - case STOP ⇒ cluster.shutdown() - - case DISCONNECT ⇒ cluster.disconnect() - - case RECONNECT ⇒ cluster.reconnect() - - case RESIGN ⇒ cluster.resign() - - case FAIL_OVER_CONNECTIONS ⇒ - val (from, to) = payloadFor(message, classOf[(InetSocketAddress, InetSocketAddress)]) - cluster.failOverClusterActorRefConnections(from, to) - - case FUNCTION_FUN0_UNIT ⇒ - localActorOf(new Actor() { - self.dispatcher = computeGridDispatcher - - def receive = { - case f: Function0[_] ⇒ try { - f() - } finally { - self.stop() - } - } - }).start ! payloadFor(message, classOf[Function0[Unit]]) - - case FUNCTION_FUN0_ANY ⇒ - localActorOf(new Actor() { - self.dispatcher = computeGridDispatcher - - def receive = { - case f: Function0[_] ⇒ try { - self.reply(f()) - } finally { - self.stop() - } - } - }).start forward payloadFor(message, classOf[Function0[Any]]) - - case FUNCTION_FUN1_ARG_UNIT ⇒ - localActorOf(new Actor() { - self.dispatcher = computeGridDispatcher - - def receive = { - case (fun: Function[_, _], param: Any) ⇒ try { - fun.asInstanceOf[Any ⇒ Unit].apply(param) - } finally { - self.stop() - } - } - }).start ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]]) - - case FUNCTION_FUN1_ARG_ANY ⇒ - localActorOf(new Actor() { - self.dispatcher = computeGridDispatcher - - def receive = { - case (fun: Function[_, _], param: Any) ⇒ try { - self.reply(fun.asInstanceOf[Any ⇒ Any](param)) - } finally { - self.stop() - } - } - }).start forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]]) + case USE ⇒ handleUse(message) + case RELEASE ⇒ handleRelease(message) + case START ⇒ cluster.start() + case STOP ⇒ cluster.shutdown() + case DISCONNECT ⇒ cluster.disconnect() + case RECONNECT ⇒ cluster.reconnect() + case RESIGN ⇒ cluster.resign() + case FAIL_OVER_CONNECTIONS ⇒ handleFailover(message) + case FUNCTION_FUN0_UNIT ⇒ handle_fun0_unit(message) + case FUNCTION_FUN0_ANY ⇒ handle_fun0_any(message) + case FUNCTION_FUN1_ARG_UNIT ⇒ handle_fun1_arg_unit(message) + case FUNCTION_FUN1_ARG_ANY ⇒ handle_fun1_arg_any(message) + //TODO: should we not deal with unrecognized message types? } case unknown ⇒ EventHandler.warning(this, "Unknown message [%s]".format(unknown)) } + def handleRelease(message: ClusterProtocol.RemoteDaemonMessageProtocol) { + if (message.hasActorUuid) { + cluster.actorAddressForUuid(uuidProtocolToUuid(message.getActorUuid)) foreach { address ⇒ + cluster.release(address) + } + } else if (message.hasActorAddress) { + cluster release message.getActorAddress + } else { + EventHandler.warning(this, + "None of 'uuid' or 'actorAddress'' is specified, ignoring remote cluster daemon command [%s]".format(message)) + } + } + + def handleUse(message: ClusterProtocol.RemoteDaemonMessageProtocol) { + def deserializeMessages(entriesAsBytes: Vector[Array[Byte]]): Vector[AnyRef] = { + import akka.remote.protocol.RemoteProtocol._ + import akka.remote.MessageSerializer + + entriesAsBytes map { bytes ⇒ + val messageBytes = + if (Cluster.shouldCompressData) LZF.uncompress(bytes) + else bytes + MessageSerializer.deserialize(MessageProtocol.parseFrom(messageBytes), None) + } + } + + def createActorRefToUseForReplay(snapshotAsBytes: Option[Array[Byte]], actorAddress: String, newActorRef: LocalActorRef): ActorRef = { + snapshotAsBytes match { + + // we have a new actor ref - the snapshot + case Some(bytes) ⇒ + // stop the new actor ref and use the snapshot instead + //TODO: What if that actor already has been retrieved and is being used?? + //So do we have a race here? + cluster.remoteService.unregister(actorAddress) + + // deserialize the snapshot actor ref and register it as remote actor + val uncompressedBytes = + if (Cluster.shouldCompressData) LZF.uncompress(bytes) + else bytes + + val snapshotActorRef = fromBinary(uncompressedBytes, newActorRef.uuid).start() + cluster.remoteService.register(actorAddress, snapshotActorRef) + + // FIXME we should call 'stop()' here (to GC the actor), but can't since that will currently + //shut down the TransactionLog for this UUID - since both this actor and the new snapshotActorRef + //have the same UUID (which they should) + //newActorRef.stop() + + snapshotActorRef + + // we have no snapshot - use the new actor ref + case None ⇒ + newActorRef + } + } + + try { + if (message.hasActorAddress) { + val actorAddress = message.getActorAddress + cluster.serializerForActor(actorAddress) foreach { serializer ⇒ + cluster.use(actorAddress, serializer) foreach { newActorRef ⇒ + cluster.remoteService.register(actorAddress, newActorRef) + + if (message.hasReplicateActorFromUuid) { + // replication is used - fetch the messages and replay them + val replicateFromUuid = uuidProtocolToUuid(message.getReplicateActorFromUuid) + val deployment = Deployer.deploymentFor(actorAddress) + val replicationScheme = DeploymentConfig.replicationSchemeFor(deployment).getOrElse( + throw new IllegalStateException( + "Actor [" + actorAddress + "] should have been configured as a replicated actor but could not find its ReplicationScheme")) + val isWriteBehind = DeploymentConfig.isWriteBehindReplication(replicationScheme) + + try { + // get the transaction log for the actor UUID + val readonlyTxLog = TransactionLog.logFor(replicateFromUuid.toString, isWriteBehind, replicationScheme) + + // get the latest snapshot (Option[Array[Byte]]) and all the subsequent messages (Array[Byte]) + val (snapshotAsBytes, entriesAsBytes) = readonlyTxLog.latestSnapshotAndSubsequentEntries + + // deserialize and restore actor snapshot. This call will automatically recreate a transaction log. + val actorRef = createActorRefToUseForReplay(snapshotAsBytes, actorAddress, newActorRef) + + // deserialize the messages + val messages: Vector[AnyRef] = deserializeMessages(entriesAsBytes) + + EventHandler.info(this, "Replaying [%s] messages to actor [%s]".format(messages.size, actorAddress)) + + // replay all messages + messages foreach { message ⇒ + EventHandler.debug(this, "Replaying message [%s] to actor [%s]".format(message, actorAddress)) + + // FIXME how to handle '?' messages? + // We can *not* replay them with the correct semantics. Should we: + // 1. Ignore/drop them and log warning? + // 2. Throw exception when about to log them? + // 3. Other? + actorRef ! message + } + + } catch { + case e: Throwable ⇒ + EventHandler.error(e, this, e.toString) + throw e + } + } + } + } + } else { + EventHandler.error(this, "Actor 'address' is not defined, ignoring remote cluster daemon command [%s]".format(message)) + } + self.reply(Success) + } catch { + case error ⇒ + self.reply(Failure(error)) + throw error + } + } + + def handle_fun0_unit(message: ClusterProtocol.RemoteDaemonMessageProtocol) { + localActorOf(new Actor() { + self.dispatcher = computeGridDispatcher + + def receive = { + case f: Function0[_] ⇒ try { + f() + } finally { + self.stop() + } + } + }).start ! payloadFor(message, classOf[Function0[Unit]]) + } + + def handle_fun0_any(message: ClusterProtocol.RemoteDaemonMessageProtocol) { + localActorOf(new Actor() { + self.dispatcher = computeGridDispatcher + + def receive = { + case f: Function0[_] ⇒ try { + self.reply(f()) + } finally { + self.stop() + } + } + }).start forward payloadFor(message, classOf[Function0[Any]]) + } + + def handle_fun1_arg_unit(message: ClusterProtocol.RemoteDaemonMessageProtocol) { + localActorOf(new Actor() { + self.dispatcher = computeGridDispatcher + + def receive = { + case (fun: Function[_, _], param: Any) ⇒ try { + fun.asInstanceOf[Any ⇒ Unit].apply(param) + } finally { + self.stop() + } + } + }).start ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]]) + } + + def handle_fun1_arg_any(message: ClusterProtocol.RemoteDaemonMessageProtocol) { + localActorOf(new Actor() { + self.dispatcher = computeGridDispatcher + + def receive = { + case (fun: Function[_, _], param: Any) ⇒ try { + self.reply(fun.asInstanceOf[Any ⇒ Any](param)) + } finally { + self.stop() + } + } + }).start forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]]) + } + + def handleFailover(message: ClusterProtocol.RemoteDaemonMessageProtocol) { + val (from, to) = payloadFor(message, classOf[(InetSocketAddress, InetSocketAddress)]) + cluster.failOverClusterActorRefConnections(from, to) + } + private def payloadFor[T](message: RemoteDaemonMessageProtocol, clazz: Class[T]): T = { Serialization.deserialize(message.getPayload.toByteArray, clazz, None) match { case Left(error) ⇒ throw error diff --git a/akka-cluster/src/main/scala/akka/cluster/TransactionLog.scala b/akka-cluster/src/main/scala/akka/cluster/TransactionLog.scala index d12820c130..0b33bfe6f3 100644 --- a/akka-cluster/src/main/scala/akka/cluster/TransactionLog.scala +++ b/akka-cluster/src/main/scala/akka/cluster/TransactionLog.scala @@ -14,26 +14,25 @@ import akka.config._ import Config._ import akka.util._ import akka.actor._ -import DeploymentConfig.{ ReplicationScheme } +import DeploymentConfig.ReplicationScheme import akka.event.EventHandler import akka.dispatch.{ DefaultPromise, Promise, MessageInvocation } import akka.remote.MessageSerializer import akka.cluster.zookeeper._ -import akka.serialization.Compression -import Compression.LZF import akka.serialization.ActorSerialization._ +import akka.serialization.Compression.LZF import java.util.Enumeration // FIXME allow user to choose dynamically between 'async' and 'sync' tx logging (asyncAddEntry(byte[] data, AddCallback cb, Object ctx)) // FIXME clean up old entries in log after doing a snapshot -// FIXME clean up all meta-data in ZK for a specific UUID when the corresponding actor is shut down -// FIXME delete tx log after migration of actor has been made and create a new one /** * @author Jonas Bonér */ -class ReplicationException(message: String) extends AkkaException(message) +class ReplicationException(message: String, cause: Throwable = null) extends AkkaException(message) { + def this(msg: String) = this(msg, null); +} /** * TODO: Explain something about threadsafety. @@ -58,39 +57,50 @@ class TransactionLog private ( /** * Record an Actor message invocation. + * + * @param invocation the MessageInvocation to record + * @param actorRef the LocalActorRef that received the message. + * @throws ReplicationException if the TransactionLog already is closed. */ - def recordEntry(messageHandle: MessageInvocation, actorRef: LocalActorRef) { + def recordEntry(invocation: MessageInvocation, actorRef: LocalActorRef) { val entryId = ledger.getLastAddPushed + 1 - if (entryId != 0 && (entryId % snapshotFrequency) == 0) { - recordSnapshot(toBinary(actorRef, false, replicationScheme)) - } else recordEntry(MessageSerializer.serialize(messageHandle.message.asInstanceOf[AnyRef]).toByteArray) + val needsSnapshot = entryId != 0 && (entryId % snapshotFrequency) == 0 + + if (needsSnapshot) { + //todo: could it be that the message is never persisted when a snapshot is added? + val bytes = toBinary(actorRef, false, replicationScheme) + recordSnapshot(bytes) + } else { + val bytes = MessageSerializer.serialize(invocation.message.asInstanceOf[AnyRef]).toByteArray + recordEntry(bytes) + } } /** * Record an entry. + * + * @param entry the entry in byte form to record. + * @throws ReplicationException if the TransactionLog already is closed. */ def recordEntry(entry: Array[Byte]) { if (isOpen.isOn) { - val bytes = + val entryBytes = if (Cluster.shouldCompressData) LZF.compress(entry) else entry + try { if (isAsync) { ledger.asyncAddEntry( - bytes, + entryBytes, new AsyncCallback.AddCallback { - def addComplete( - returnCode: Int, - ledgerHandle: LedgerHandle, - entryId: Long, - ctx: AnyRef) { + def addComplete(returnCode: Int, ledgerHandle: LedgerHandle, entryId: Long, ctx: AnyRef) { handleReturnCode(returnCode) EventHandler.debug(this, "Writing entry [%s] to log [%s]".format(entryId, logId)) } }, null) } else { - handleReturnCode(ledger.addEntry(bytes)) + handleReturnCode(ledger.addEntry(entryBytes)) val entryId = ledger.getLastAddPushed EventHandler.debug(this, "Writing entry [%s] to log [%s]".format(entryId, logId)) } @@ -102,22 +112,22 @@ class TransactionLog private ( /** * Record a snapshot. + * + * @param snapshot the snapshot in byteform to record. + * @throws ReplicationException if the TransactionLog already is closed. */ def recordSnapshot(snapshot: Array[Byte]) { if (isOpen.isOn) { - val bytes = + val snapshotBytes = if (Cluster.shouldCompressData) LZF.compress(snapshot) else snapshot + try { if (isAsync) { ledger.asyncAddEntry( - bytes, + snapshotBytes, new AsyncCallback.AddCallback { - def addComplete( - returnCode: Int, - ledgerHandle: LedgerHandle, - snapshotId: Long, - ctx: AnyRef) { + def addComplete(returnCode: Int, ledgerHandle: LedgerHandle, snapshotId: Long, ctx: AnyRef) { handleReturnCode(returnCode) EventHandler.debug(this, "Writing snapshot to log [%s]".format(snapshotId)) storeSnapshotMetaDataInZooKeeper(snapshotId) @@ -125,10 +135,18 @@ class TransactionLog private ( }, null) } else { - handleReturnCode(ledger.addEntry(bytes)) + //todo: could this be racy, since writing the snapshot itself and storing the snapsnot id, is not + //an atomic operation? + + //first store the snapshot. + handleReturnCode(ledger.addEntry(snapshotBytes)) val snapshotId = ledger.getLastAddPushed + //this is the location where all previous entries can be removed. + //TODO: how to remove data? + EventHandler.debug(this, "Writing snapshot to log [%s]".format(snapshotId)) + //and now store the snapshot metadata. storeSnapshotMetaDataInZooKeeper(snapshotId) } } catch { @@ -139,6 +157,8 @@ class TransactionLog private ( /** * Get all the entries for this transaction log. + * + * @throws ReplicationException if the TransactionLog already is closed. */ def entries: Vector[Array[Byte]] = entriesInRange(0, ledger.getLastAddConfirmed) @@ -168,26 +188,26 @@ class TransactionLog private ( /** * Get a range of entries from 'from' to 'to' for this transaction log. + * + * @param from the first element of the range + * @param the last index from the range (including). + * @return a Vector containing Byte Arrays. Each element in the vector is a record. + * @throws IllegalArgumenException if from or to is negative, or if 'from' is bigger than 'to'. + * @throws ReplicationException if the TransactionLog already is closed. */ def entriesInRange(from: Long, to: Long): Vector[Array[Byte]] = if (isOpen.isOn) { try { if (from < 0) throw new IllegalArgumentException("'from' index can't be negative [" + from + "]") if (to < 0) throw new IllegalArgumentException("'to' index can't be negative [" + from + "]") if (to < from) throw new IllegalArgumentException("'to' index can't be smaller than 'from' index [" + from + "," + to + "]") - EventHandler.debug(this, - "Reading entries [%s -> %s] for log [%s]".format(from, to, logId)) + EventHandler.debug(this, "Reading entries [%s -> %s] for log [%s]".format(from, to, logId)) if (isAsync) { val future = new DefaultPromise[Vector[Array[Byte]]](timeout) ledger.asyncReadEntries( from, to, new AsyncCallback.ReadCallback { - def readComplete( - returnCode: Int, - ledgerHandle: LedgerHandle, - enumeration: Enumeration[LedgerEntry], - ctx: AnyRef) { - + def readComplete(returnCode: Int, ledgerHandle: LedgerHandle, enumeration: Enumeration[LedgerEntry], ctx: AnyRef) { val future = ctx.asInstanceOf[Promise[Vector[Array[Byte]]]] val entries = toByteArrays(enumeration) @@ -207,6 +227,8 @@ class TransactionLog private ( /** * Get the last entry written to this transaction log. + * + * Returns -1 if there has never been an entry. */ def latestEntryId: Long = ledger.getLastAddConfirmed @@ -216,8 +238,7 @@ class TransactionLog private ( def latestSnapshotId: Option[Long] = { try { val snapshotId = zkClient.readData(snapshotPath).asInstanceOf[Long] - EventHandler.debug(this, - "Retrieved latest snapshot id [%s] from transaction log [%s]".format(snapshotId, logId)) + EventHandler.debug(this, "Retrieved latest snapshot id [%s] from transaction log [%s]".format(snapshotId, logId)) Some(snapshotId) } catch { case e: ZkNoNodeException ⇒ None @@ -226,7 +247,10 @@ class TransactionLog private ( } /** - * Delete all entries for this transaction log. + * Delete this transaction log. So all entries but also all metadata will be removed. + * + * TODO: Behavior unclear what happens when already deleted (what happens to the ledger). + * TODO: Behavior unclear what happens when already closed. */ def delete() { if (isOpen.isOn) { @@ -244,6 +268,10 @@ class TransactionLog private ( } else { bookieClient.deleteLedger(logId) } + + //also remote everything else that belongs to this TransactionLog. + zkClient.delete(snapshotPath) + zkClient.delete(txLogPath) } catch { case e ⇒ handleError(e) } @@ -252,6 +280,8 @@ class TransactionLog private ( /** * Close this transaction log. + * + * If already closed, the call is ignored. */ def close() { if (isOpen.switchOff) { @@ -303,8 +333,7 @@ class TransactionLog private ( } catch { case e ⇒ handleError(new ReplicationException( - "Could not store transaction log snapshot meta-data in ZooKeeper for UUID [" + - id + "]")) + "Could not store transaction log snapshot meta-data in ZooKeeper for UUID [" + id + "]")) } EventHandler.debug(this, "Writing snapshot [%s] to log [%s]".format(snapshotId, logId)) } else transactionClosedError @@ -398,18 +427,32 @@ object TransactionLog { } /** - * Creates a new transaction log for the 'id' specified. + * Checks if a TransactionLog for the given id already exists. */ - def newLogFor( - id: String, - isAsync: Boolean, - replicationScheme: ReplicationScheme): TransactionLog = { + def exists(id: String): Boolean = { + val txLogPath = transactionLogNode + "/" + id + zkClient.exists(txLogPath) + } + /** + * Creates a new transaction log for the 'id' specified. If a TransactionLog already exists for the id, + * it will be overwritten. + */ + def newLogFor(id: String, isAsync: Boolean, replicationScheme: ReplicationScheme): TransactionLog = { val txLogPath = transactionLogNode + "/" + id val ledger = try { - if (zkClient.exists(txLogPath)) throw new ReplicationException( - "Transaction log for UUID [" + id + "] already exists") + if (exists(id)) { + //if it exists, we need to delete it first. This gives it the overwrite semantics we are looking for. + try { + val ledger = bookieClient.createLedger(ensembleSize, quorumSize, digestType, password) + val txLog = TransactionLog(ledger, id, false, null) + txLog.delete() + txLog.close() + } catch { + case e ⇒ handleError(e) + } + } val future = new DefaultPromise[LedgerHandle](timeout) if (isAsync) { @@ -438,13 +481,13 @@ object TransactionLog { try { zkClient.create(txLogPath, null, CreateMode.PERSISTENT) zkClient.writeData(txLogPath, logId) - logId + logId //TODO: does this have any effect? } catch { case e ⇒ bookieClient.deleteLedger(logId) // clean up handleError(new ReplicationException( "Could not store transaction log [" + logId + - "] meta-data in ZooKeeper for UUID [" + id + "]")) + "] meta-data in ZooKeeper for UUID [" + id + "]", e)) } EventHandler.info(this, "Created new transaction log [%s] for UUID [%s]".format(logId, id)) @@ -453,12 +496,10 @@ object TransactionLog { /** * Fetches an existing transaction log for the 'id' specified. + * + * @throws ReplicationException if the log with the given id doesn't exist. */ - def logFor( - id: String, - isAsync: Boolean, - replicationScheme: ReplicationScheme): TransactionLog = { - + def logFor(id: String, isAsync: Boolean, replicationScheme: ReplicationScheme): TransactionLog = { val txLogPath = transactionLogNode + "/" + id val logId = try { @@ -479,10 +520,7 @@ object TransactionLog { bookieClient.asyncOpenLedger( logId, digestType, password, new AsyncCallback.OpenCallback { - def openComplete( - returnCode: Int, - ledgerHandle: LedgerHandle, - ctx: AnyRef) { + def openComplete(returnCode: Int, ledgerHandle: LedgerHandle, ctx: AnyRef) { val future = ctx.asInstanceOf[Promise[LedgerHandle]] if (returnCode == BKException.Code.OK) future.completeWithResult(ledgerHandle) else future.completeWithException(BKException.create(returnCode)) @@ -514,20 +552,29 @@ object TransactionLog { } /** + * TODO: Documentation. + * * @author Jonas Bonér */ object LocalBookKeeperEnsemble { private val isRunning = new Switch(false) + + //TODO: should probably come from the config file. private val port = 5555 @volatile private var localBookKeeper: LocalBookKeeper = _ /** - * TODO document method + * Starts the LocalBookKeeperEnsemble. + * + * Call can safely be made when already started. + * + * This call will block until it is started. */ def start() { isRunning switchOn { + EventHandler.info(this, "Starting LocalBookKeeperEnsemble") localBookKeeper = new LocalBookKeeper(TransactionLog.ensembleSize) localBookKeeper.runZookeeper(port) localBookKeeper.initializeZookeper() @@ -537,7 +584,11 @@ object LocalBookKeeperEnsemble { } /** - * TODO document method + * Shuts down the LocalBookKeeperEnsemble. + * + * Call can safely bemade when already shutdown. + * + * This call will block until the shutdown completes. */ def shutdown() { isRunning switchOff { diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterTestNode.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterTestNode.scala index bdc430ee6d..a3d8c44fe7 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterTestNode.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/ClusterTestNode.scala @@ -13,28 +13,6 @@ import akka.util.Duration import System.{ currentTimeMillis ⇒ now } import java.io.File -import akka.actor.Deployer - -trait MasterClusterTestNode extends WordSpec with MustMatchers with BeforeAndAfterAll { - def testNodes: Int - - override def beforeAll() = { - Cluster.startLocalCluster() - onReady() - ClusterTestNode.ready(getClass.getName) - } - - def onReady() = {} - - override def afterAll() = { - ClusterTestNode.waitForExits(getClass.getName, testNodes - 1) - ClusterTestNode.cleanUp(getClass.getName) - onShutdown() - Cluster.shutdownLocalCluster() - } - - def onShutdown() = {} -} trait ClusterTestNode extends WordSpec with MustMatchers with BeforeAndAfterAll { diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/MasterClusterTestNode.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/MasterClusterTestNode.scala new file mode 100644 index 0000000000..0d2b078d11 --- /dev/null +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/MasterClusterTestNode.scala @@ -0,0 +1,31 @@ +/** + * Copyright (C) 2009-2011 Typesafe Inc. + */ + +package akka.cluster + +import org.scalatest.WordSpec +import org.scalatest.matchers.MustMatchers +import org.scalatest.BeforeAndAfterAll + +trait MasterClusterTestNode extends WordSpec with MustMatchers with BeforeAndAfterAll { + def testNodes: Int + + override def beforeAll() = { + Cluster.startLocalCluster() + onReady() + ClusterTestNode.ready(getClass.getName) + } + + def onReady() = {} + + override def afterAll() = { + ClusterTestNode.waitForExits(getClass.getName, testNodes - 1) + ClusterTestNode.cleanUp(getClass.getName) + onShutdown() + Cluster.shutdownLocalCluster() + } + + def onShutdown() = {} +} + diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmSpec.scala index c40a06b404..1138cb7f46 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmSpec.scala @@ -47,8 +47,7 @@ class ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmNode1 extends Cluste for (i ← 0 until 10) (actorRef ? Count(i)).as[String] must be(Some("World from node [node1]")) } - barrier("start-node2", NrOfNodes) { - } + barrier("start-node2", NrOfNodes).await() node.shutdown() } @@ -64,11 +63,9 @@ class ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmNode2 extends Master "be able to replicate an actor with a transaction log and replay transaction log after actor migration" in { - barrier("start-node1", NrOfNodes) { - } + barrier("start-node1", NrOfNodes).await() - barrier("create-actor-on-node1", NrOfNodes) { - } + barrier("create-actor-on-node1", NrOfNodes).await() barrier("start-node2", NrOfNodes) { node.start() diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/snapshot/ReplicationTransactionLogWriteBehindSnapshotMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/snapshot/ReplicationTransactionLogWriteBehindSnapshotMultiJvmSpec.scala index 80ad04a4df..bdb920451e 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/snapshot/ReplicationTransactionLogWriteBehindSnapshotMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writebehind/snapshot/ReplicationTransactionLogWriteBehindSnapshotMultiJvmSpec.scala @@ -68,8 +68,7 @@ class ReplicationTransactionLogWriteBehindSnapshotMultiJvmNode1 extends ClusterT (actorRef ? Count(counter)).as[String].get must be("World from node [node1]") } - barrier("start-node2", NrOfNodes) { - } + barrier("start-node2", NrOfNodes).await() node.shutdown() } @@ -85,11 +84,9 @@ class ReplicationTransactionLogWriteBehindSnapshotMultiJvmNode2 extends MasterCl "be able to replicate an actor with a transaction log and replay transaction log after actor migration" in { - barrier("start-node1", NrOfNodes) { - } + barrier("start-node1", NrOfNodes).await() - barrier("create-actor-on-node1", NrOfNodes) { - } + barrier("create-actor-on-node1", NrOfNodes).await() barrier("start-node2", NrOfNodes) { node.start() diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmNode1.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmNode1.conf index 42e57847b5..74957902ed 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmNode1.conf +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmNode1.conf @@ -1,8 +1,6 @@ akka.enabled-modules = ["cluster"] akka.event-handler-level = "WARNING" akka.actor.deployment.hello-world.router = "direct" -akka.actor.deployment.hello-world.clustered.replication-factor = 1 - akka.actor.deployment.hello-world.clustered.replication.storage = "transaction-log" akka.actor.deployment.hello-world.clustered.replication.strategy = "write-through" akka.cluster.replication.snapshot-frequency = 1000 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmNode2.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmNode2.conf index cc2fb1ef3b..74957902ed 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmNode2.conf +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteBehindNoSnapshotMultiJvmNode2.conf @@ -1,7 +1,6 @@ akka.enabled-modules = ["cluster"] akka.event-handler-level = "WARNING" akka.actor.deployment.hello-world.router = "direct" -akka.actor.deployment.hello-world.clustered.replication-factor = 1 akka.actor.deployment.hello-world.clustered.replication.storage = "transaction-log" akka.actor.deployment.hello-world.clustered.replication.strategy = "write-through" akka.cluster.replication.snapshot-frequency = 1000 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmNode1.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmNode1.conf index cc2fb1ef3b..74957902ed 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmNode1.conf +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmNode1.conf @@ -1,7 +1,6 @@ akka.enabled-modules = ["cluster"] akka.event-handler-level = "WARNING" akka.actor.deployment.hello-world.router = "direct" -akka.actor.deployment.hello-world.clustered.replication-factor = 1 akka.actor.deployment.hello-world.clustered.replication.storage = "transaction-log" akka.actor.deployment.hello-world.clustered.replication.strategy = "write-through" akka.cluster.replication.snapshot-frequency = 1000 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmNode2.conf b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmNode2.conf index cc2fb1ef3b..74957902ed 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmNode2.conf +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmNode2.conf @@ -1,7 +1,6 @@ akka.enabled-modules = ["cluster"] akka.event-handler-level = "WARNING" akka.actor.deployment.hello-world.router = "direct" -akka.actor.deployment.hello-world.clustered.replication-factor = 1 akka.actor.deployment.hello-world.clustered.replication.storage = "transaction-log" akka.actor.deployment.hello-world.clustered.replication.strategy = "write-through" akka.cluster.replication.snapshot-frequency = 1000 diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmSpec.scala index 97fbb1c79b..f1f5a4d71b 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/nosnapshot/ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmSpec.scala @@ -20,9 +20,11 @@ object ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmSpec { var log = "" def receive = { case Count(nr) ⇒ + println("Received number: " + nr) log += nr.toString self.reply("World from node [" + Config.nodename + "]") case GetLog ⇒ + println("Received getLog") self.reply(Log(log)) } } @@ -47,8 +49,7 @@ class ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmNode1 extends Clust (actorRef ? Count(i)).as[String] must be(Some("World from node [node1]")) } - barrier("start-node2", NrOfNodes) { - } + barrier("start-node2", NrOfNodes).await() node.shutdown() } @@ -64,11 +65,9 @@ class ReplicationTransactionLogWriteThroughNoSnapshotMultiJvmNode2 extends Maste "be able to replicate an actor with a transaction log and replay transaction log after actor migration" in { - barrier("start-node1", NrOfNodes) { - } + barrier("start-node1", NrOfNodes).await() - barrier("create-actor-on-node1", NrOfNodes) { - } + barrier("create-actor-on-node1", NrOfNodes).await() barrier("start-node2", NrOfNodes) { node.start() diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/snapshot/ReplicationTransactionLogWriteThroughSnapshotMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/snapshot/ReplicationTransactionLogWriteThroughSnapshotMultiJvmSpec.scala index b1136a5490..96ca99f074 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/snapshot/ReplicationTransactionLogWriteThroughSnapshotMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/replication/transactionlog/writethrough/snapshot/ReplicationTransactionLogWriteThroughSnapshotMultiJvmSpec.scala @@ -68,8 +68,7 @@ class ReplicationTransactionLogWriteThroughSnapshotMultiJvmNode1 extends Cluster (actorRef ? Count(counter)).as[String].get must be("World from node [node1]") } - barrier("start-node2", NrOfNodes) { - } + barrier("start-node2", NrOfNodes).await() node.shutdown() } @@ -85,11 +84,9 @@ class ReplicationTransactionLogWriteThroughSnapshotMultiJvmNode2 extends MasterC "be able to replicate an actor with a transaction log and replay transaction log after actor migration" in { - barrier("start-node1", NrOfNodes) { - } + barrier("start-node1", NrOfNodes).await() - barrier("create-actor-on-node1", NrOfNodes) { - } + barrier("create-actor-on-node1", NrOfNodes).await() barrier("start-node2", NrOfNodes) { node.start() diff --git a/akka-cluster/src/test/scala/akka/cluster/TransactionLogSpec.scala b/akka-cluster/src/test/scala/akka/cluster/TransactionLogSpec.scala index 259c5179e1..b35d0b3d49 100644 --- a/akka-cluster/src/test/scala/akka/cluster/TransactionLogSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/TransactionLogSpec.scala @@ -3,33 +3,47 @@ */ package akka.cluster -import org.apache.bookkeeper.client.{ BookKeeper, BKException } -import BKException._ -import org.apache.zookeeper.server.ZooKeeperServer - +import org.apache.bookkeeper.client.BookKeeper import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers -import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach, Spec } - -import akka.serialization._ -import akka.actor._ -import ActorSerialization._ -import Actor._ - -import java.util.concurrent.{ CyclicBarrier, TimeUnit } -import java.io.File -import java.nio.ByteBuffer +import org.scalatest.BeforeAndAfterAll import com.eaio.uuid.UUID -import scala.collection.JavaConversions._ - class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterAll { private var bookKeeper: BookKeeper = _ private var localBookKeeper: LocalBookKeeper = _ // synchronous API - "A Transaction Log" should { + "A synchronous Transaction Log" should { + + "be able to be deleted - synchronous" in { + val uuid = (new UUID).toString + val txlog = TransactionLog.newLogFor(uuid, false, null) + val entry = "hello".getBytes("UTF-8") + txlog.recordEntry(entry) + + txlog.delete() + txlog.close() + + val zkClient = TransactionLog.zkClient + assert(zkClient.readData(txlog.snapshotPath, true) == null) + assert(zkClient.readData(txlog.txLogPath, true) == null) + } + + "fail to be opened if non existing - synchronous" in { + val uuid = (new UUID).toString + intercept[ReplicationException](TransactionLog.logFor(uuid, false, null)) + } + + "be able to be checked for existence - synchronous" in { + val uuid = (new UUID).toString + TransactionLog.exists(uuid) must be(false) + + TransactionLog.newLogFor(uuid, false, null) + TransactionLog.exists(uuid) must be(true) + } + "be able to record entries - synchronous" in { val uuid = (new UUID).toString val txlog = TransactionLog.newLogFor(uuid, false, null) @@ -37,6 +51,19 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA txlog.recordEntry(entry) } + "be able to overweite an existing txlog if one already exists - synchronous" in { + val uuid = (new UUID).toString + val txlog1 = TransactionLog.newLogFor(uuid, false, null) + val entry = "hello".getBytes("UTF-8") + txlog1.recordEntry(entry) + txlog1.recordEntry(entry) + txlog1.close + + val txLog2 = TransactionLog.newLogFor(uuid, false, null) + txLog2.latestSnapshotId.isDefined must be(false) + txLog2.latestEntryId must be(-1) + } + "be able to record and delete entries - synchronous" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, false, null) @@ -45,7 +72,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA txlog1.recordEntry(entry) txlog1.delete txlog1.close - intercept[BKNoSuchLedgerExistsException](TransactionLog.logFor(uuid, false, null)) + intercept[ReplicationException](TransactionLog.logFor(uuid, false, null)) } "be able to record entries and read entries with 'entriesInRange' - synchronous" in { @@ -146,7 +173,7 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA } } - "A Transaction Log" should { + "An asynchronous Transaction Log" should { "be able to record entries - asynchronous" in { val uuid = (new UUID).toString val txlog = TransactionLog.newLogFor(uuid, true, null) @@ -156,6 +183,46 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA txlog.close } + "be able to be deleted - asynchronous" in { + val uuid = (new UUID).toString + val txlog = TransactionLog.newLogFor(uuid, true, null) + val entry = "hello".getBytes("UTF-8") + txlog.recordEntry(entry) + + txlog.delete() + txlog.close() + + val zkClient = TransactionLog.zkClient + assert(zkClient.readData(txlog.snapshotPath, true) == null) + assert(zkClient.readData(txlog.txLogPath, true) == null) + } + + "be able to be checked for existence - asynchronous" in { + val uuid = (new UUID).toString + TransactionLog.exists(uuid) must be(false) + + TransactionLog.newLogFor(uuid, true, null) + TransactionLog.exists(uuid) must be(true) + } + + "fail to be opened if non existing - asynchronous" in { + val uuid = (new UUID).toString + intercept[ReplicationException](TransactionLog.logFor(uuid, true, null)) + } + + "be able to overweite an existing txlog if one already exists - asynchronous" in { + val uuid = (new UUID).toString + val txlog1 = TransactionLog.newLogFor(uuid, true, null) + val entry = "hello".getBytes("UTF-8") + txlog1.recordEntry(entry) + txlog1.recordEntry(entry) + txlog1.close + + val txLog2 = TransactionLog.newLogFor(uuid, true, null) + txLog2.latestSnapshotId.isDefined must be(false) + txLog2.latestEntryId must be(-1) + } + "be able to record and delete entries - asynchronous" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, true, null) @@ -167,8 +234,9 @@ class TransactionLogSpec extends WordSpec with MustMatchers with BeforeAndAfterA Thread.sleep(200) txlog1.delete Thread.sleep(200) - intercept[BKNoSuchLedgerExistsException](TransactionLog.logFor(uuid, true, null)) + intercept[ReplicationException](TransactionLog.logFor(uuid, true, null)) } + "be able to record entries and read entries with 'entriesInRange' - asynchronous" in { val uuid = (new UUID).toString val txlog1 = TransactionLog.newLogFor(uuid, true, null) From 0ae7a72f3aa7b38c97166a664cb010090583aeb6 Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Sat, 6 Aug 2011 14:10:36 -0600 Subject: [PATCH 38/53] Future: Reschedule onTimeout/orElse if not yet expired --- .../src/main/scala/akka/dispatch/Future.scala | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index cfd729a25e..3876634628 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -816,7 +816,14 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi if (!timeout.duration.isFinite) false //Not possible else if (value.isEmpty) { if (!isExpired) { - val runnable = new Runnable { def run() { if (!isCompleted) func(DefaultPromise.this) } } //TODO Reschedule is run prematurely + val runnable = new Runnable { + def run() { + if (!isCompleted) { + if (!isExpired) Scheduler.scheduleOnce(this, timeLeft, NANOS) + else func(DefaultPromise.this) + } + } + } Scheduler.scheduleOnce(runnable, timeLeft, NANOS) false } else true @@ -837,7 +844,10 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi promise completeWith this val runnable = new Runnable { def run() { - if (!isCompleted) promise complete (try { Right(fallback) } catch { case e: Exception ⇒ Left(e) }) + if (!isCompleted) { + if (!isExpired) Scheduler.scheduleOnce(this, timeLeft, NANOS) + else promise complete (try { Right(fallback) } catch { case e: Exception ⇒ Left(e) }) + } } } Scheduler.scheduleOnce(runnable, timeLeft, NANOS) From a17b75fc09409d684a379a395e216d061712b72a Mon Sep 17 00:00:00 2001 From: Derek Williams Date: Sat, 6 Aug 2011 15:00:15 -0600 Subject: [PATCH 39/53] Add check for jdk7 to disable -optimize --- project/AkkaBuild.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index bd30d3b4fd..7a1fb598d6 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -298,7 +298,8 @@ object AkkaBuild extends Build { resolvers += "Twitter Public Repo" at "http://maven.twttr.com", // This will be going away with com.mongodb.async's next release // compile options - scalacOptions ++= Seq("-encoding", "UTF-8", "-optimise", "-deprecation", "-unchecked"), + scalacOptions ++= Seq("-encoding", "UTF-8", "-deprecation", "-unchecked") ++ ( + if (System getProperty "java.runtime.version" startsWith "1.7") Seq() else Seq("-optimize")), // -optimize fails with jdk7 javacOptions ++= Seq("-Xlint:unchecked", "-Xlint:deprecation"), // add config dir to classpaths From a5ca2bad155ded81151d35a01ce7478d6c270891 Mon Sep 17 00:00:00 2001 From: Peter Veentjer Date: Sun, 7 Aug 2011 14:47:45 +0300 Subject: [PATCH 40/53] ticket #958, resolved review comments --- akka-actor/src/main/scala/akka/routing/Routing.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index a0d5eadbb0..aa4e19d69f 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -4,9 +4,6 @@ package akka.routing -import akka.actor.Actor._ -import scala.collection.JavaConversions._ -import scala.collection.immutable.Seq import java.util.concurrent.atomic.AtomicReference import annotation.tailrec From 8a9e13e0ffc312ce6260db2b50db8d20f878f85a Mon Sep 17 00:00:00 2001 From: Peter Veentjer Date: Sun, 7 Aug 2011 16:15:04 +0300 Subject: [PATCH 41/53] ticket 934: fixed review comments --- .../src/main/scala/akka/cluster/ClusterActorRef.scala | 4 ++-- .../Random3ReplicasMultiJvmSpec.scala | 11 ++++++++--- .../RoundRobin1ReplicaMultiJvmSpec.scala | 7 ++----- 3 files changed, 12 insertions(+), 10 deletions(-) diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala index 6745eb1370..c3d4986a62 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala @@ -58,7 +58,7 @@ class ClusterActorRef private[akka] (inetSocketAddresses: Array[Tuple2[UUID, Ine } private[akka] def failOver(from: InetSocketAddress, to: InetSocketAddress): Unit = { - EventHandler.info(this, "ClusterActorRef. %s failover from %s to %s".format(address, from, to)) + EventHandler.debug(this, "ClusterActorRef. %s failover from %s to %s".format(address, from, to)) @tailrec def doFailover(from: InetSocketAddress, to: InetSocketAddress): Unit = { @@ -96,7 +96,7 @@ class ClusterActorRef private[akka] (inetSocketAddresses: Array[Tuple2[UUID, Ine } def signalDeadActor(ref: ActorRef): Unit = { - EventHandler.info(this, "ClusterActorRef %s signalDeadActor %s".format(address, ref.address)) + EventHandler.debug(this, "ClusterActorRef %s signalDeadActor %s".format(address, ref.address)) //since the number remote actor refs for a clustered actor ref is quite low, we can deal with the O(N) complexity //of the following removal. diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmSpec.scala index 9bc162ab52..91b8caeb0b 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/random/replicationfactor_3/Random3ReplicasMultiJvmSpec.scala @@ -80,9 +80,14 @@ class Random3ReplicasMultiJvmNode2 extends ClusterTestNode { count((hello ? "Hello").as[String].getOrElse(fail("Should have recieved reply from a node"))) } - assert(replies("World from node [node1]") > 100) - assert(replies("World from node [node2]") > 100) - assert(replies("World from node [node3]") > 100) + val repliesNode1 = replies("World from node [node1]") + val repliesNode2 = replies("World from node [node2]") + val repliesNode3 = replies("World from node [node3]") + + assert(repliesNode1 > 100) + assert(repliesNode2 > 100) + assert(repliesNode3 > 100) + assert(repliesNode1 + repliesNode2 + repliesNode3 === 1000) barrier("end", NrOfNodes).await() diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_1/RoundRobin1ReplicaMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_1/RoundRobin1ReplicaMultiJvmSpec.scala index bc33d0c078..9f42cb19aa 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_1/RoundRobin1ReplicaMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/roundrobin/replicationfactor_1/RoundRobin1ReplicaMultiJvmSpec.scala @@ -4,14 +4,9 @@ package akka.cluster.routing.roundrobin.replicationfactor_1 -import org.scalatest.WordSpec -import org.scalatest.matchers.MustMatchers -import org.scalatest.BeforeAndAfterAll - import akka.cluster._ import Cluster._ import akka.actor._ -import akka.actor.Actor._ import akka.config.Config /** @@ -25,9 +20,11 @@ object RoundRobin1ReplicaMultiJvmSpec { self.reply("World from node [" + Config.nodename + "]") } } + } class RoundRobin1ReplicaMultiJvmNode1 extends MasterClusterTestNode { + import RoundRobin1ReplicaMultiJvmSpec._ val testNodes = 1 From 4cd4917f9fd1972e349ba4e54fc3e1bb706cebc4 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Sun, 7 Aug 2011 19:23:25 +0200 Subject: [PATCH 42/53] Turning remote compression off by default, closing ticket #1083 --- akka-cluster/src/main/scala/akka/remote/RemoteConfig.scala | 2 +- config/akka-reference.conf | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/akka-cluster/src/main/scala/akka/remote/RemoteConfig.scala b/akka-cluster/src/main/scala/akka/remote/RemoteConfig.scala index c54e2b4a9f..23896fe0e7 100644 --- a/akka-cluster/src/main/scala/akka/remote/RemoteConfig.scala +++ b/akka-cluster/src/main/scala/akka/remote/RemoteConfig.scala @@ -34,7 +34,7 @@ object RemoteServerSettings { val UNTRUSTED_MODE = config.getBool("akka.cluster.server.untrusted-mode", false) val PORT = config.getInt("akka.cluster.server.port", 2552) val CONNECTION_TIMEOUT_MILLIS = Duration(config.getInt("akka.cluster.server.connection-timeout", 1), TIME_UNIT) - val COMPRESSION_SCHEME = config.getString("akka.cluster.compression-scheme", "zlib") + val COMPRESSION_SCHEME = config.getString("akka.cluster.compression-scheme", "") val ZLIB_COMPRESSION_LEVEL = { val level = config.getInt("akka.cluster.zlib-compression-level", 6) if (level < 1 && level > 9) throw new IllegalArgumentException( diff --git a/config/akka-reference.conf b/config/akka-reference.conf index 218a824220..ac0ebcf50c 100644 --- a/config/akka-reference.conf +++ b/config/akka-reference.conf @@ -190,7 +190,7 @@ akka { remote-daemon-ack-timeout = 30 # Timeout for ACK of cluster operations, lik checking actor out etc. include-ref-node-in-replica-set = on # Can a replica be instantiated on the same node as the cluster reference to the actor # Default: on - compression-scheme = "zlib" # Options: "zlib" (lzf to come), leave out for no compression + compression-scheme = "" # Options: "zlib" (lzf to come), leave out for no compression zlib-compression-level = 6 # Options: 0-9 (1 being fastest and 9 being the most compressed), default is 6 # FIXME rename to transport From e433eda2c2a73832e93ae79b4fc3c8fc9a8311ec Mon Sep 17 00:00:00 2001 From: Peter Veentjer Date: Mon, 8 Aug 2011 09:11:39 +0300 Subject: [PATCH 43/53] fix for a failing test:who changes the compression to none but didn't fix the testsgit add -A..you badgit add -A --- .../src/test/scala/akka/actor/actor/ClusterSpec.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/actor/ClusterSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/actor/ClusterSpec.scala index 9569e1bed5..2bec70e6f5 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/actor/ClusterSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/actor/ClusterSpec.scala @@ -6,7 +6,7 @@ import akka.config.Config class ClusterSpec extends WordSpec with MustMatchers { - "A Deployer" must { + "ClusterSpec: A Deployer" must { "be able to parse 'akka.actor.cluster._' config elements" in { import Config.config._ @@ -21,7 +21,7 @@ class ClusterSpec extends WordSpec with MustMatchers { getInt("akka.cluster.connection-timeout") must equal(Some(60)) getInt("akka.cluster.remote-daemon-ack-timeout") must equal(Some(30)) getBool("akka.cluster.include-ref-node-in-replica-set") must equal(Some(true)) - getString("akka.cluster.compression-scheme") must equal(Some("zlib")) + getString("akka.cluster.compression-scheme") must equal(Some("")) getInt("akka.cluster.zlib-compression-level") must equal(Some(6)) getString("akka.cluster.layer") must equal(Some("akka.remote.netty.NettyRemoteSupport")) getString("akka.cluster.secure-cookie") must equal(Some("")) From b8af8df41f522f866f75a2b7e5ae696c2eebef12 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 8 Aug 2011 08:39:22 +0200 Subject: [PATCH 44/53] Fixing ClusterSpec --- .../src/test/scala/akka/actor/actor/ClusterSpec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/actor/ClusterSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/actor/ClusterSpec.scala index 9569e1bed5..4805737ec9 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/actor/ClusterSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/actor/ClusterSpec.scala @@ -21,7 +21,7 @@ class ClusterSpec extends WordSpec with MustMatchers { getInt("akka.cluster.connection-timeout") must equal(Some(60)) getInt("akka.cluster.remote-daemon-ack-timeout") must equal(Some(30)) getBool("akka.cluster.include-ref-node-in-replica-set") must equal(Some(true)) - getString("akka.cluster.compression-scheme") must equal(Some("zlib")) + getString("akka.cluster.compression-scheme") must equal(Some("")) getInt("akka.cluster.zlib-compression-level") must equal(Some(6)) getString("akka.cluster.layer") must equal(Some("akka.remote.netty.NettyRemoteSupport")) getString("akka.cluster.secure-cookie") must equal(Some("")) From 5c4488730f39ba8175487673bf4f5cb312e7cc78 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 8 Aug 2011 15:38:22 +0200 Subject: [PATCH 45/53] Fixing FutureTimeoutException so that it has a String constructor so it's deserializable in remoting --- akka-actor/src/main/scala/akka/dispatch/Future.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 3876634628..50e1266ee5 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -25,7 +25,9 @@ import akka.util.{ Switch, Duration, BoxedType } import java.util.concurrent.atomic.{ AtomicReference, AtomicBoolean } import scala.Math -class FutureTimeoutException(message: String, cause: Throwable = null) extends AkkaException(message, cause) +class FutureTimeoutException(message: String, cause: Throwable = null) extends AkkaException(message, cause) { + def this(message: String) = this(message, null) +} object Futures { From e3bfd2f31a7006b7a95e20cd411feed616f79bcd Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 8 Aug 2011 16:53:54 +0200 Subject: [PATCH 46/53] Removing the shutdownHook from the Actor object since we are no longer using Configgy and hence no risk of leaks thereof --- .../src/main/scala/akka/actor/Actor.scala | 19 +------------------ .../src/main/scala/akka/util/AkkaLoader.scala | 1 - 2 files changed, 1 insertion(+), 19 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index 7fec89b4c2..d00ca86adb 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -162,7 +162,7 @@ object Timeout { * * @author Jonas Bonér */ -object Actor extends ListenerManagement { +object Actor { /** * A Receive is a convenience type that defines actor message behavior currently modeled as @@ -170,23 +170,6 @@ object Actor extends ListenerManagement { */ type Receive = PartialFunction[Any, Unit] - /** - * Add shutdown cleanups - */ - private[akka] lazy val shutdownHook = { - val hook = new Runnable { - override def run() { - // Clear Thread.subclassAudits - val tf = classOf[java.lang.Thread].getDeclaredField("subclassAudits") - tf.setAccessible(true) - val subclassAudits = tf.get(null).asInstanceOf[java.util.Map[_, _]] - subclassAudits synchronized { subclassAudits.clear() } - } - } - Runtime.getRuntime.addShutdownHook(new Thread(hook, "akka-shutdown-hook")) - hook - } - private[actor] val actorRefInCreation = new ThreadLocal[Stack[ActorRef]] { override def initialValue = Stack[ActorRef]() } diff --git a/akka-actor/src/main/scala/akka/util/AkkaLoader.scala b/akka-actor/src/main/scala/akka/util/AkkaLoader.scala index 7057ecc561..d3ff9bce32 100644 --- a/akka-actor/src/main/scala/akka/util/AkkaLoader.scala +++ b/akka-actor/src/main/scala/akka/util/AkkaLoader.scala @@ -38,7 +38,6 @@ class AkkaLoader { println("Shutting down Akka...") _bundles.foreach(_.onUnload) _bundles = None - Actor.shutdownHook.run println("Akka succesfully shut down") } } From e3107715c1f8a75bcc6320ca8188e1e40a2f91bc Mon Sep 17 00:00:00 2001 From: Peter Veentjer Date: Mon, 8 Aug 2011 18:43:30 +0300 Subject: [PATCH 47/53] Fix of failing ClusterActorRefCleanupMultiJvmSpec --- .../ClusterActorRefCleanupMultiJvmSpec.scala | 131 ++++++++++-------- 1 file changed, 70 insertions(+), 61 deletions(-) diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala index fc4b2b3f5e..ea1095439e 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala @@ -41,71 +41,80 @@ class ClusterActorRefCleanupMultiJvmNode1 extends MasterClusterTestNode { "ClusterActorRef" must { "cleanup itself" in { - node.start - barrier("awaitStarted", NrOfNodes).await() - - val ref = Actor.actorOf[ClusterActorRefCleanupMultiJvmSpec.TestActor]("service-test") - - ref.isInstanceOf[ClusterActorRef] must be(true) - - val clusteredRef = ref.asInstanceOf[ClusterActorRef] - - //verify that all remote actors are there. - clusteredRef.connections.size must be(2) - - // ignore exceptions from killing nodes - val ignoreExceptions = Seq( - EventFilter[ClosedChannelException], - EventFilter[NotYetConnectedException], - EventFilter[RoutingException], - EventFilter[ConnectException]) - - EventHandler.notify(TestEvent.Mute(ignoreExceptions)) - - //let one of the actors die. - clusteredRef ! "Die" - - //just some waiting to make sure that the node has died. - Thread.sleep(5000) - - //send some request, this should trigger the cleanup try { + + node.start + barrier("awaitStarted", NrOfNodes).await() + + val ref = Actor.actorOf[ClusterActorRefCleanupMultiJvmSpec.TestActor]("service-test") + + ref.isInstanceOf[ClusterActorRef] must be(true) + + val clusteredRef = ref.asInstanceOf[ClusterActorRef] + + //verify that all remote actors are there. + clusteredRef.connections.size must be(2) + + // ignore exceptions from killing nodes + val ignoreExceptions = Seq( + EventFilter[ClosedChannelException], + EventFilter[NotYetConnectedException], + EventFilter[RoutingException], + EventFilter[ConnectException]) + + EventHandler.notify(TestEvent.Mute(ignoreExceptions)) + + //let one of the actors die. + clusteredRef ! "Die" + + //just some waiting to make sure that the node has died. + Thread.sleep(5000) + + //send some request, this should trigger the cleanup + try { + clusteredRef ! "hello" + clusteredRef ! "hello" + } catch { + case e: ClosedChannelException ⇒ + case e: NotYetConnectedException ⇒ + case e: RoutingException ⇒ + } + + //since the call to the node failed, the node must have been removed from the list. + clusteredRef.connections.size must be(1) + + //send a message to this node, clusteredRef ! "hello" - clusteredRef ! "hello" + + //now kill another node + clusteredRef ! "Die" + + //just some waiting to make sure that the node has died. + Thread.sleep(5000) + + //trigger the cleanup. + try { + clusteredRef ! "hello" + } catch { + case e: ClosedChannelException ⇒ + case e: NotYetConnectedException ⇒ + case e: RoutingException ⇒ + } + + //now there must not be any remaining connections after the dead of the last actor. + clusteredRef.connections.size must be(0) + + //and lets make sure we now get the correct exception if we try to use the ref. + intercept[RoutingException] { + clusteredRef ! "Hello" + } + + node.shutdown() } catch { - case e: NotYetConnectedException ⇒ + case e: Throwable ⇒ + e.printStackTrace(); + throw e } - - //since the call to the node failed, the node must have been removed from the list. - clusteredRef.connections.size must be(1) - - //send a message to this node, - clusteredRef ! "hello" - - //now kill another node - clusteredRef ! "Die" - - //just some waiting to make sure that the node has died. - Thread.sleep(5000) - - //trigger the cleanup. - try { - clusteredRef ! "hello" - } catch { - case e: ClosedChannelException ⇒ - case e: NotYetConnectedException ⇒ - case e: RoutingException ⇒ - } - - //now there must not be any remaining connections after the dead of the last actor. - clusteredRef.connections.size must be(0) - - //and lets make sure we now get the correct exception if we try to use the ref. - intercept[RoutingException] { - clusteredRef ! "Hello" - } - - node.shutdown() } } } From 74b425fdc7ecdc00b16c8027c1faedaa7c7a2650 Mon Sep 17 00:00:00 2001 From: Peter Veentjer Date: Mon, 8 Aug 2011 18:45:16 +0300 Subject: [PATCH 48/53] Fix of failing ClusterActorRefCleanupMultiJvmSpec: also removed some debugging code --- .../reflogic/ClusterActorRefCleanupMultiJvmSpec.scala | 7 ------- 1 file changed, 7 deletions(-) diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala index ea1095439e..5db131211b 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala @@ -41,8 +41,6 @@ class ClusterActorRefCleanupMultiJvmNode1 extends MasterClusterTestNode { "ClusterActorRef" must { "cleanup itself" in { - try { - node.start barrier("awaitStarted", NrOfNodes).await() @@ -110,11 +108,6 @@ class ClusterActorRefCleanupMultiJvmNode1 extends MasterClusterTestNode { } node.shutdown() - } catch { - case e: Throwable ⇒ - e.printStackTrace(); - throw e - } } } } From bd049718be180751bc11e7397fa13ff5b30dc6fe Mon Sep 17 00:00:00 2001 From: Peter Veentjer Date: Mon, 8 Aug 2011 20:40:18 +0300 Subject: [PATCH 49/53] ticket #992: misc fixes for transaction log, processed review comments --- .../src/main/scala/akka/cluster/Cluster.scala | 6 +-- .../scala/akka/cluster/TransactionLog.scala | 42 ++++++++++--------- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala index 128c8c1864..7bc612f097 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala @@ -1800,9 +1800,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor { def handleRelease(message: ClusterProtocol.RemoteDaemonMessageProtocol) { if (message.hasActorUuid) { - cluster.actorAddressForUuid(uuidProtocolToUuid(message.getActorUuid)) foreach { address ⇒ - cluster.release(address) - } + cluster.actorAddressForUuid(uuidProtocolToUuid(message.getActorUuid)) foreach cluster.release(_) } else if (message.hasActorAddress) { cluster release message.getActorAddress } else { @@ -1911,7 +1909,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor { } self.reply(Success) } catch { - case error ⇒ + case error:Throwable ⇒ self.reply(Failure(error)) throw error } diff --git a/akka-cluster/src/main/scala/akka/cluster/TransactionLog.scala b/akka-cluster/src/main/scala/akka/cluster/TransactionLog.scala index 0b33bfe6f3..0988714875 100644 --- a/akka-cluster/src/main/scala/akka/cluster/TransactionLog.scala +++ b/akka-cluster/src/main/scala/akka/cluster/TransactionLog.scala @@ -31,7 +31,7 @@ import java.util.Enumeration * @author Jonas Bonér */ class ReplicationException(message: String, cause: Throwable = null) extends AkkaException(message) { - def this(msg: String) = this(msg, null); + def this(msg: String) = this(msg, null) } /** @@ -50,7 +50,7 @@ class TransactionLog private ( import TransactionLog._ val logId = ledger.getId - val txLogPath = transactionLogNode + "/" + id + val txLogPath = transactionLogPath(id) val snapshotPath = txLogPath + "/snapshot" private val isOpen = new Switch(true) @@ -105,7 +105,7 @@ class TransactionLog private ( EventHandler.debug(this, "Writing entry [%s] to log [%s]".format(entryId, logId)) } } catch { - case e ⇒ handleError(e) + case e: Throwable ⇒ handleError(e) } } else transactionClosedError } @@ -150,7 +150,7 @@ class TransactionLog private ( storeSnapshotMetaDataInZooKeeper(snapshotId) } } catch { - case e ⇒ handleError(e) + case e: Throwable ⇒ handleError(e) } } else transactionClosedError } @@ -221,7 +221,7 @@ class TransactionLog private ( toByteArrays(ledger.readEntries(from, to)) } } catch { - case e ⇒ handleError(e) + case e: Throwable ⇒ handleError(e) } } else transactionClosedError @@ -242,7 +242,7 @@ class TransactionLog private ( Some(snapshotId) } catch { case e: ZkNoNodeException ⇒ None - case e ⇒ handleError(e) + case e: Throwable ⇒ handleError(e) } } @@ -273,7 +273,7 @@ class TransactionLog private ( zkClient.delete(snapshotPath) zkClient.delete(txLogPath) } catch { - case e ⇒ handleError(e) + case e: Throwable ⇒ handleError(e) } } } @@ -302,7 +302,7 @@ class TransactionLog private ( ledger.close() } } catch { - case e ⇒ handleError(e) + case e: Throwable ⇒ handleError(e) } } } @@ -325,13 +325,13 @@ class TransactionLog private ( zkClient.create(snapshotPath, null, CreateMode.PERSISTENT) } catch { case e: ZkNodeExistsException ⇒ {} // do nothing - case e ⇒ handleError(e) + case e: Throwable ⇒ handleError(e) } try { zkClient.writeData(snapshotPath, snapshotId) } catch { - case e ⇒ + case e: Throwable ⇒ handleError(new ReplicationException( "Could not store transaction log snapshot meta-data in ZooKeeper for UUID [" + id + "]")) } @@ -386,7 +386,7 @@ object TransactionLog { zk.create(transactionLogNode, null, CreateMode.PERSISTENT) } catch { case e: ZkNodeExistsException ⇒ {} // do nothing - case e ⇒ handleError(e) + case e: Throwable ⇒ handleError(e) } EventHandler.info(this, @@ -421,16 +421,18 @@ object TransactionLog { bookieClient.halt() EventHandler.info(this, "Transaction log shut down successfully") } catch { - case e ⇒ handleError(e) + case e: Throwable ⇒ handleError(e) } } } + def transactionLogPath(id: String): String = transactionLogNode + "/" + id + /** * Checks if a TransactionLog for the given id already exists. */ def exists(id: String): Boolean = { - val txLogPath = transactionLogNode + "/" + id + val txLogPath = transactionLogPath(id) zkClient.exists(txLogPath) } @@ -439,7 +441,7 @@ object TransactionLog { * it will be overwritten. */ def newLogFor(id: String, isAsync: Boolean, replicationScheme: ReplicationScheme): TransactionLog = { - val txLogPath = transactionLogNode + "/" + id + val txLogPath = transactionLogPath(id) val ledger = try { if (exists(id)) { @@ -450,7 +452,7 @@ object TransactionLog { txLog.delete() txLog.close() } catch { - case e ⇒ handleError(e) + case e: Throwable ⇒ handleError(e) } } @@ -474,7 +476,7 @@ object TransactionLog { bookieClient.createLedger(ensembleSize, quorumSize, digestType, password) } } catch { - case e ⇒ handleError(e) + case e: Throwable ⇒ handleError(e) } val logId = ledger.getId @@ -483,7 +485,7 @@ object TransactionLog { zkClient.writeData(txLogPath, logId) logId //TODO: does this have any effect? } catch { - case e ⇒ + case e: Throwable ⇒ bookieClient.deleteLedger(logId) // clean up handleError(new ReplicationException( "Could not store transaction log [" + logId + @@ -500,7 +502,7 @@ object TransactionLog { * @throws ReplicationException if the log with the given id doesn't exist. */ def logFor(id: String, isAsync: Boolean, replicationScheme: ReplicationScheme): TransactionLog = { - val txLogPath = transactionLogNode + "/" + id + val txLogPath = transactionLogPath(id) val logId = try { val logId = zkClient.readData(txLogPath).asInstanceOf[Long] @@ -511,7 +513,7 @@ object TransactionLog { case e: ZkNoNodeException ⇒ handleError(new ReplicationException( "Transaction log for UUID [" + id + "] does not exist in ZooKeeper")) - case e ⇒ handleError(e) + case e: Throwable ⇒ handleError(e) } val ledger = try { @@ -532,7 +534,7 @@ object TransactionLog { bookieClient.openLedger(logId, digestType, password) } } catch { - case e ⇒ handleError(e) + case e: Throwable ⇒ handleError(e) } TransactionLog(ledger, id, isAsync, replicationScheme) From c8e938a44cf86dc78c6c6dcaac89dc149bae9bcb Mon Sep 17 00:00:00 2001 From: Peter Veentjer Date: Mon, 8 Aug 2011 20:57:01 +0300 Subject: [PATCH 50/53] ticket #992: misc fixes for transaction log, processed review comments --- .../src/main/scala/akka/cluster/Cluster.scala | 6 +- .../ClusterActorRefCleanupMultiJvmSpec.scala | 102 +++++++++--------- 2 files changed, 55 insertions(+), 53 deletions(-) diff --git a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala index 7bc612f097..6a39806a31 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala @@ -1800,7 +1800,9 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor { def handleRelease(message: ClusterProtocol.RemoteDaemonMessageProtocol) { if (message.hasActorUuid) { - cluster.actorAddressForUuid(uuidProtocolToUuid(message.getActorUuid)) foreach cluster.release(_) + cluster.actorAddressForUuid(uuidProtocolToUuid(message.getActorUuid)) foreach { address ⇒ + cluster.release(address) + } } else if (message.hasActorAddress) { cluster release message.getActorAddress } else { @@ -1909,7 +1911,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor { } self.reply(Success) } catch { - case error:Throwable ⇒ + case error: Throwable ⇒ self.reply(Failure(error)) throw error } diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala index 5db131211b..1308d89a47 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/reflogic/ClusterActorRefCleanupMultiJvmSpec.scala @@ -41,73 +41,73 @@ class ClusterActorRefCleanupMultiJvmNode1 extends MasterClusterTestNode { "ClusterActorRef" must { "cleanup itself" in { - node.start - barrier("awaitStarted", NrOfNodes).await() + node.start + barrier("awaitStarted", NrOfNodes).await() - val ref = Actor.actorOf[ClusterActorRefCleanupMultiJvmSpec.TestActor]("service-test") + val ref = Actor.actorOf[ClusterActorRefCleanupMultiJvmSpec.TestActor]("service-test") - ref.isInstanceOf[ClusterActorRef] must be(true) + ref.isInstanceOf[ClusterActorRef] must be(true) - val clusteredRef = ref.asInstanceOf[ClusterActorRef] + val clusteredRef = ref.asInstanceOf[ClusterActorRef] - //verify that all remote actors are there. - clusteredRef.connections.size must be(2) + //verify that all remote actors are there. + clusteredRef.connections.size must be(2) - // ignore exceptions from killing nodes - val ignoreExceptions = Seq( - EventFilter[ClosedChannelException], - EventFilter[NotYetConnectedException], - EventFilter[RoutingException], - EventFilter[ConnectException]) + // ignore exceptions from killing nodes + val ignoreExceptions = Seq( + EventFilter[ClosedChannelException], + EventFilter[NotYetConnectedException], + EventFilter[RoutingException], + EventFilter[ConnectException]) - EventHandler.notify(TestEvent.Mute(ignoreExceptions)) + EventHandler.notify(TestEvent.Mute(ignoreExceptions)) - //let one of the actors die. - clusteredRef ! "Die" + //let one of the actors die. + clusteredRef ! "Die" - //just some waiting to make sure that the node has died. - Thread.sleep(5000) + //just some waiting to make sure that the node has died. + Thread.sleep(5000) - //send some request, this should trigger the cleanup - try { - clusteredRef ! "hello" - clusteredRef ! "hello" - } catch { - case e: ClosedChannelException ⇒ - case e: NotYetConnectedException ⇒ - case e: RoutingException ⇒ - } - - //since the call to the node failed, the node must have been removed from the list. - clusteredRef.connections.size must be(1) - - //send a message to this node, + //send some request, this should trigger the cleanup + try { clusteredRef ! "hello" + clusteredRef ! "hello" + } catch { + case e: ClosedChannelException ⇒ + case e: NotYetConnectedException ⇒ + case e: RoutingException ⇒ + } - //now kill another node - clusteredRef ! "Die" + //since the call to the node failed, the node must have been removed from the list. + clusteredRef.connections.size must be(1) - //just some waiting to make sure that the node has died. - Thread.sleep(5000) + //send a message to this node, + clusteredRef ! "hello" - //trigger the cleanup. - try { - clusteredRef ! "hello" - } catch { - case e: ClosedChannelException ⇒ - case e: NotYetConnectedException ⇒ - case e: RoutingException ⇒ - } + //now kill another node + clusteredRef ! "Die" - //now there must not be any remaining connections after the dead of the last actor. - clusteredRef.connections.size must be(0) + //just some waiting to make sure that the node has died. + Thread.sleep(5000) - //and lets make sure we now get the correct exception if we try to use the ref. - intercept[RoutingException] { - clusteredRef ! "Hello" - } + //trigger the cleanup. + try { + clusteredRef ! "hello" + } catch { + case e: ClosedChannelException ⇒ + case e: NotYetConnectedException ⇒ + case e: RoutingException ⇒ + } - node.shutdown() + //now there must not be any remaining connections after the dead of the last actor. + clusteredRef.connections.size must be(0) + + //and lets make sure we now get the correct exception if we try to use the ref. + intercept[RoutingException] { + clusteredRef ! "Hello" + } + + node.shutdown() } } } From f1bc34cad72ac323694c0c667a143bc245ba51d6 Mon Sep 17 00:00:00 2001 From: Peter Veentjer Date: Tue, 9 Aug 2011 13:19:59 +0300 Subject: [PATCH 51/53] ticket 989 #Allow specifying preferred-nodes of other type than node:/ currently the task has been downscoped to only allow node:name since we need to rethink this part. Metadata is going to make it / it possible to deal with way more nodes --- .../src/main/scala/akka/actor/Deployer.scala | 6 +++--- .../main/scala/akka/actor/DeploymentConfig.scala | 16 ++++++++-------- config/akka-reference.conf | 6 +++--- 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index 3e98716722..63bdcbe826 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -21,7 +21,7 @@ import akka.AkkaException */ object Deployer { - val defaultAddress = Host(Config.hostname) +// val defaultAddress = Node(Config.nodename) lazy val instance: ClusterModule.ClusterDeployer = { val deployer = @@ -191,9 +191,9 @@ object Deployer { val address = tokenizer.nextElement.asInstanceOf[String] protocol match { - case "host" ⇒ Host(address) + //case "host" ⇒ Host(address) case "node" ⇒ Node(address) - case "ip" ⇒ IP(address) + //case "ip" ⇒ IP(address) case _ ⇒ raiseHomeConfigError() } } diff --git a/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala b/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala index f5206b7668..8f3b4504ce 100644 --- a/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala +++ b/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala @@ -51,7 +51,7 @@ object DeploymentConfig { // -------------------------------- sealed trait Scope case class Clustered( - preferredNodes: Iterable[Home] = Vector(Host("localhost")), + preferredNodes: Iterable[Home] = Vector(Node(Config.nodename)), replicas: ReplicationFactor = ZeroReplicationFactor, replication: ReplicationScheme = Transient) extends Scope @@ -65,9 +65,9 @@ object DeploymentConfig { // --- Home // -------------------------------- sealed trait Home - case class Host(hostName: String) extends Home +// case class Host(hostName: String) extends Home case class Node(nodeName: String) extends Home - case class IP(ipAddress: String) extends Home +// case class IP(ipAddress: String) extends Home // -------------------------------- // --- Replicas @@ -130,11 +130,11 @@ object DeploymentConfig { def nodeNameFor(home: Home): String = home match { case Node(nodename) ⇒ nodename - case Host("localhost") ⇒ Config.nodename - case IP("0.0.0.0") ⇒ Config.nodename - case IP("127.0.0.1") ⇒ Config.nodename - case Host(hostname) ⇒ throw new UnsupportedOperationException("Specifying preferred node name by 'hostname' is not yet supported. Use the node name like: preferred-nodes = [\"node:node1\"]") - case IP(address) ⇒ throw new UnsupportedOperationException("Specifying preferred node name by 'IP address' is not yet supported. Use the node name like: preferred-nodes = [\"node:node1\"]") +// case Host("localhost") ⇒ Config.nodename +// case IP("0.0.0.0") ⇒ Config.nodename +// case IP("127.0.0.1") ⇒ Config.nodename +// case Host(hostname) ⇒ throw new UnsupportedOperationException("Specifying preferred node name by 'hostname' is not yet supported. Use the node name like: preferred-nodes = [\"node:node1\"]") +// case IP(address) ⇒ throw new UnsupportedOperationException("Specifying preferred node name by 'IP address' is not yet supported. Use the node name like: preferred-nodes = [\"node:node1\"]") } def isHomeNode(homes: Iterable[Home]): Boolean = homes exists (home ⇒ nodeNameFor(home) == Config.nodename) diff --git a/config/akka-reference.conf b/config/akka-reference.conf index 218a824220..cbd2e5113e 100644 --- a/config/akka-reference.conf +++ b/config/akka-reference.conf @@ -51,9 +51,9 @@ akka { # default (if omitted) is local non-clustered actor preferred-nodes = ["node:node1"] # a list of preferred nodes for instantiating the actor instances on - # defined as: hostname, IP-address or node name - # available: "host:", "ip:" and "node:" - # default is "host:localhost" + # defined as node name + # available: + # "node:" replication-factor = 3 # number of actor instances in the cluster # available: positive integer (0-N) or the string "auto" for auto-scaling From 6a49efd966bf372e6a6d1f3e5500a9d5017bf14c Mon Sep 17 00:00:00 2001 From: Peter Veentjer Date: Tue, 9 Aug 2011 13:33:10 +0300 Subject: [PATCH 52/53] ticket 989 #Allow specifying preferred-nodes of other type than node:/ currently the task has been downscoped to only allow node:name since we need to rethink this part. Metadata is going to make it / it possible to deal with way more nodes --- .../src/main/scala/akka/actor/Deployer.scala | 2 +- .../main/scala/akka/actor/DeploymentConfig.scala | 16 ++++++++-------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index 5a5834d8a6..818bd09de9 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -31,7 +31,7 @@ trait ActorDeployer { */ object Deployer extends ActorDeployer { -// val defaultAddress = Node(Config.nodename) + // val defaultAddress = Node(Config.nodename) lazy val instance: ActorDeployer = { val deployer = if (ClusterModule.isEnabled) ClusterModule.clusterDeployer else LocalDeployer diff --git a/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala b/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala index 5ee5b0a870..ff539f57b8 100644 --- a/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala +++ b/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala @@ -73,9 +73,9 @@ object DeploymentConfig { // --- Home // -------------------------------- sealed trait Home -// case class Host(hostName: String) extends Home + // case class Host(hostName: String) extends Home case class Node(nodeName: String) extends Home -// case class IP(ipAddress: String) extends Home + // case class IP(ipAddress: String) extends Home // -------------------------------- // --- Replicas @@ -137,12 +137,12 @@ object DeploymentConfig { // -------------------------------- def nodeNameFor(home: Home): String = home match { - case Node(nodename) ⇒ nodename -// case Host("localhost") ⇒ Config.nodename -// case IP("0.0.0.0") ⇒ Config.nodename -// case IP("127.0.0.1") ⇒ Config.nodename -// case Host(hostname) ⇒ throw new UnsupportedOperationException("Specifying preferred node name by 'hostname' is not yet supported. Use the node name like: preferred-nodes = [\"node:node1\"]") -// case IP(address) ⇒ throw new UnsupportedOperationException("Specifying preferred node name by 'IP address' is not yet supported. Use the node name like: preferred-nodes = [\"node:node1\"]") + case Node(nodename) ⇒ nodename + // case Host("localhost") ⇒ Config.nodename + // case IP("0.0.0.0") ⇒ Config.nodename + // case IP("127.0.0.1") ⇒ Config.nodename + // case Host(hostname) ⇒ throw new UnsupportedOperationException("Specifying preferred node name by 'hostname' is not yet supported. Use the node name like: preferred-nodes = [\"node:node1\"]") + // case IP(address) ⇒ throw new UnsupportedOperationException("Specifying preferred node name by 'IP address' is not yet supported. Use the node name like: preferred-nodes = [\"node:node1\"]") } def isHomeNode(homes: Iterable[Home]): Boolean = homes exists (home ⇒ nodeNameFor(home) == Config.nodename) From 5eb15409e60e9977553b5bfb8df0b6abcc442442 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 9 Aug 2011 19:31:51 +0200 Subject: [PATCH 53/53] Adding DeployId and LocalOnly to Props --- .../src/main/scala/akka/actor/ActorRef.scala | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 55b8a4e4d5..243a1e9149 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -65,12 +65,14 @@ object Props { * ActorRef configuration object, this is thread safe and fully sharable */ case class Props(creator: () ⇒ Actor, + deployId: String = "", dispatcher: MessageDispatcher = Props.defaultDispatcher, timeout: Timeout = Props.defaultTimeout, receiveTimeout: Option[Duration] = None, lifeCycle: LifeCycle = Permanent, faultHandler: FaultHandlingStrategy = NoFaultHandlingStrategy, - supervisor: Option[ActorRef] = None) { + supervisor: Option[ActorRef] = None, + localOnly: Boolean = false) { /** * Returns a new Props with the specified creator set * Scala API @@ -83,6 +85,12 @@ case class Props(creator: () ⇒ Actor, */ def withCreator(c: Creator[Actor]) = copy(creator = () ⇒ c.create) + /** + * Returns a new Props with the specified deployId set + * Java and Scala API + */ + def withDeployId(id: String) = copy(deployId = if (id eq null) "" else id) + /** * Returns a new Props with the specified dispatcher set * Java API @@ -142,6 +150,12 @@ case class Props(creator: () ⇒ Actor, * Scala API */ def withReceiveTimeout(r: scala.Option[Duration]) = copy(receiveTimeout = r) + + /** + * Returns a new Props with the specified localOnly set + * Java and Scala API + */ + def withLocalOnly(l: Boolean) = copy(localOnly = l) } /**