From 613f63b5266c249da814e48cecee7b7270dcd87b Mon Sep 17 00:00:00 2001 From: Roland Kuhn Date: Fri, 30 Jan 2015 11:33:33 +0100 Subject: [PATCH 1/4] fix warnings in actor and testkit --- .../src/main/scala/akka/actor/ActorRef.scala | 2 +- .../main/scala/akka/actor/FaultHandling.scala | 16 ++++++------- .../main/scala/akka/actor/TypedActor.scala | 2 +- .../scala/akka/pattern/CircuitBreaker.scala | 2 +- .../akka/pattern/GracefulStopSupport.scala | 23 ++++++++----------- .../scala/akka/testkit/AkkaSpecSpec.scala | 9 ++++---- .../scala/akka/testkit/TestActorRefSpec.scala | 5 ++-- project/AkkaBuild.scala | 8 +++++-- 8 files changed, 34 insertions(+), 33 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 7c0a443140..f14b73c992 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -326,7 +326,7 @@ private[akka] class LocalActorRef private[akka] ( * If this method returns true, it will never return false again, but if it * returns false, you cannot be sure if it's alive still (race condition) */ - @deprecated("Use context.watch(actor) and receive Terminated(actor)", "2.2") override def isTerminated: Boolean = actorCell.isTerminated + override def isTerminated: Boolean = actorCell.isTerminated /** * Starts the actor after initialization. diff --git a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala index 5b4938695a..0f59703618 100644 --- a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala @@ -294,20 +294,20 @@ abstract class SupervisorStrategy { def handleFailure(context: ActorContext, child: ActorRef, cause: Throwable, stats: ChildRestartStats, children: Iterable[ChildRestartStats]): Boolean = { val directive = decider.applyOrElse(cause, escalateDefault) directive match { - case d @ Resume ⇒ - logFailure(context, child, cause, d) + case Resume ⇒ + logFailure(context, child, cause, directive) resumeChild(child, cause) true - case d @ Restart ⇒ - logFailure(context, child, cause, d) + case Restart ⇒ + logFailure(context, child, cause, directive) processFailure(context, true, child, cause, stats, children) true - case d @ Stop ⇒ - logFailure(context, child, cause, d) + case Stop ⇒ + logFailure(context, child, cause, directive) processFailure(context, false, child, cause, stats, children) true - case d @ Escalate ⇒ - logFailure(context, child, cause, d) + case Escalate ⇒ + logFailure(context, child, cause, directive) false } } diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index af0b41ab25..f163cf56d9 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -293,7 +293,7 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi } } - protected def withContext[T](unitOfWork: ⇒ T): T = { + protected def withContext[U](unitOfWork: ⇒ U): U = { TypedActor.selfReference set proxyVar.get TypedActor.currentContext set context try unitOfWork finally { diff --git a/akka-actor/src/main/scala/akka/pattern/CircuitBreaker.scala b/akka-actor/src/main/scala/akka/pattern/CircuitBreaker.scala index b7c2022844..ca9c9f07c5 100644 --- a/akka-actor/src/main/scala/akka/pattern/CircuitBreaker.scala +++ b/akka-actor/src/main/scala/akka/pattern/CircuitBreaker.scala @@ -300,7 +300,7 @@ class CircuitBreaker(scheduler: Scheduler, maxFailures: Int, callTimeout: Finite */ def callThrough[T](body: ⇒ Future[T]): Future[T] = { - def materialize[T](value: ⇒ Future[T]): Future[T] = try value catch { case NonFatal(t) ⇒ Future.failed(t) } + def materialize[U](value: ⇒ Future[U]): Future[U] = try value catch { case NonFatal(t) ⇒ Future.failed(t) } if (callTimeout == Duration.Zero) { materialize(body) diff --git a/akka-actor/src/main/scala/akka/pattern/GracefulStopSupport.scala b/akka-actor/src/main/scala/akka/pattern/GracefulStopSupport.scala index 14ce942334..1575d64476 100644 --- a/akka-actor/src/main/scala/akka/pattern/GracefulStopSupport.scala +++ b/akka-actor/src/main/scala/akka/pattern/GracefulStopSupport.scala @@ -46,18 +46,15 @@ trait GracefulStopSupport { * }}} */ def gracefulStop(target: ActorRef, timeout: FiniteDuration, stopMessage: Any = PoisonPill): Future[Boolean] = { - if (target.isTerminated) Future successful true - else { - val internalTarget = target.asInstanceOf[InternalActorRef] - val ref = PromiseActorRef(internalTarget.provider, Timeout(timeout), target, stopMessage.getClass.getName) - internalTarget.sendSystemMessage(Watch(internalTarget, ref)) - target.tell(stopMessage, Actor.noSender) - ref.result.future.transform( - { - case Terminated(t) if t.path == target.path ⇒ true - case _ ⇒ { internalTarget.sendSystemMessage(Unwatch(target, ref)); false } - }, - t ⇒ { internalTarget.sendSystemMessage(Unwatch(target, ref)); t })(ref.internalCallingThreadExecutionContext) - } + val internalTarget = target.asInstanceOf[InternalActorRef] + val ref = PromiseActorRef(internalTarget.provider, Timeout(timeout), target, stopMessage.getClass.getName) + internalTarget.sendSystemMessage(Watch(internalTarget, ref)) + target.tell(stopMessage, Actor.noSender) + ref.result.future.transform( + { + case Terminated(t) if t.path == target.path ⇒ true + case _ ⇒ { internalTarget.sendSystemMessage(Unwatch(target, ref)); false } + }, + t ⇒ { internalTarget.sendSystemMessage(Unwatch(target, ref)); t })(ref.internalCallingThreadExecutionContext) } } diff --git a/akka-testkit/src/test/scala/akka/testkit/AkkaSpecSpec.scala b/akka-testkit/src/test/scala/akka/testkit/AkkaSpecSpec.scala index 7495fd0172..12f3bd272e 100644 --- a/akka-testkit/src/test/scala/akka/testkit/AkkaSpecSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/AkkaSpecSpec.scala @@ -39,12 +39,11 @@ class AkkaSpecSpec extends WordSpec with Matchers { "akka.actor.debug.lifecycle" -> true, "akka.actor.debug.event-stream" -> true, "akka.loglevel" -> "DEBUG", "akka.stdout-loglevel" -> "DEBUG") val system = ActorSystem("AkkaSpec1", ConfigFactory.parseMap(conf.asJava).withFallback(AkkaSpec.testConf)) - val spec = new AkkaSpec(system) { - val ref = Seq(testActor, system.actorOf(Props.empty, "name")) - } - spec.ref foreach (_.isTerminated should not be true) + var refs = Seq.empty[ActorRef] + val spec = new AkkaSpec(system) { refs = Seq(testActor, system.actorOf(Props.empty, "name")) } + refs foreach (_.isTerminated should not be true) TestKit.shutdownActorSystem(system) - spec.awaitCond(spec.ref forall (_.isTerminated), 2 seconds) + spec.awaitCond(refs forall (_.isTerminated), 2 seconds) } "stop correctly when sending PoisonPill to rootGuardian" in { diff --git a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala index c88031530a..6ce5865e73 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala @@ -227,12 +227,13 @@ class TestActorRefSpec extends AkkaSpec("disp1.type=Dispatcher") with BeforeAndA "A TestActorRef" must { "allow access to internals" in { - val ref = TestActorRef(new TActor { + class TA extends TActor { var s: String = _ def receiveT = { case x: String ⇒ s = x } - }) + } + val ref = TestActorRef(new TA) ref ! "hallo" val actor = ref.underlyingActor actor.s should be("hallo") diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 2c6dec9ac8..f0850e4893 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -287,11 +287,15 @@ object AkkaBuild extends Build { pomIncludeRepository := (_ => false) // do not leak internal repositories during staging ) + private def deprecation: Boolean = System.getProperty("akka.deprecation", "false").toBoolean + lazy val defaultSettings = baseSettings ++ resolverSettings ++ TestExtras.Filter.settings ++ Protobuf.settings ++ Seq( // compile options - scalacOptions in Compile ++= Seq("-encoding", "UTF-8", "-target:jvm-1.6", "-deprecation", "-feature", "-unchecked", "-Xlog-reflective-calls", "-Xlint"), - javacOptions in compile ++= Seq("-encoding", "UTF-8", "-source", "1.6", "-target", "1.6", "-Xlint:unchecked", "-Xlint:deprecation"), + scalacOptions in Compile ++= Seq("-encoding", "UTF-8", "-target:jvm-1.6", "-feature", "-unchecked", "-Xlog-reflective-calls", "-Xlint"), + scalacOptions in Compile ++= (if (deprecation) Seq("-deprecation") else Nil), + javacOptions in compile ++= Seq("-encoding", "UTF-8", "-source", "1.6", "-target", "1.6", "-Xlint:unchecked", "-XDignore.symbol.file"), + javacOptions in compile ++= (if (deprecation) Seq("-Xlint:deprecation") else Nil), javacOptions in doc ++= Seq("-encoding", "UTF-8", "-source", "1.6"), incOptions := incOptions.value.withNameHashing(true), From 15ebe8f0823be989d0ae65f2cf23ad85d4dd4dcd Mon Sep 17 00:00:00 2001 From: Roland Kuhn Date: Fri, 30 Jan 2015 16:05:36 +0100 Subject: [PATCH 2/4] fix warnings in akka-actor-tests --- .../src/test/scala/akka/TestUtils.scala | 2 + .../ActorFireForgetRequestReplySpec.scala | 96 ------------------- .../scala/akka/actor/ActorMailboxSpec.scala | 8 +- .../test/scala/akka/actor/UidClashTest.scala | 8 +- .../test/scala/akka/dispatch/FutureSpec.scala | 62 ++++++------ .../scala/akka/event/EventStreamSpec.scala | 12 +-- .../test/scala/akka/event/LoggerSpec.scala | 5 +- .../akka/serialization/SerializeSpec.scala | 14 +-- project/AkkaBuild.scala | 2 + 9 files changed, 59 insertions(+), 150 deletions(-) delete mode 100644 akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala diff --git a/akka-actor-tests/src/test/scala/akka/TestUtils.scala b/akka-actor-tests/src/test/scala/akka/TestUtils.scala index da3732af9f..6bcdaaa0af 100644 --- a/akka-actor-tests/src/test/scala/akka/TestUtils.scala +++ b/akka-actor-tests/src/test/scala/akka/TestUtils.scala @@ -11,6 +11,8 @@ import java.nio.channels.{ DatagramChannel, ServerSocketChannel } import akka.actor.{ ActorSystem, ActorRef } import akka.testkit.TestProbe +import language.reflectiveCalls + object TestUtils { // Structural type needed since DatagramSocket and ServerSocket has no common ancestor apart from Object diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala deleted file mode 100644 index c51372400d..0000000000 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala +++ /dev/null @@ -1,96 +0,0 @@ -/** - * Copyright (C) 2009-2014 Typesafe Inc. - */ - -package akka.actor - -import akka.testkit._ -import org.scalatest.BeforeAndAfterEach -import scala.concurrent.duration._ -import scala.concurrent.Await -import akka.pattern.ask - -object ActorFireForgetRequestReplySpec { - - class ReplyActor extends Actor { - def receive = { - case "Send" ⇒ - sender() ! "Reply" - case "SendImplicit" ⇒ - sender() ! "ReplyImplicit" - } - } - - class CrashingActor extends Actor { - import context.system - def receive = { - case "Die" ⇒ - state.finished.await - throw new Exception("Expected exception") - } - } - - class SenderActor(replyActor: ActorRef) extends Actor { - import context.system - def receive = { - case "Init" ⇒ - replyActor ! "Send" - case "Reply" ⇒ { - state.s = "Reply" - state.finished.await - } - case "InitImplicit" ⇒ replyActor ! "SendImplicit" - case "ReplyImplicit" ⇒ { - state.s = "ReplyImplicit" - state.finished.await - } - } - } - - object state { - var s = "NIL" - val finished = TestBarrier(2) - } -} - -@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) -class ActorFireForgetRequestReplySpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout { - import ActorFireForgetRequestReplySpec._ - - override def beforeEach() = { - state.finished.reset - } - - "An Actor" must { - - "reply to bang message using reply" in { - val replyActor = system.actorOf(Props[ReplyActor]) - val senderActor = system.actorOf(Props(new SenderActor(replyActor))) - senderActor ! "Init" - state.finished.await - state.s should be("Reply") - } - - "reply to bang message using implicit sender" in { - val replyActor = system.actorOf(Props[ReplyActor]) - val senderActor = system.actorOf(Props(new SenderActor(replyActor))) - senderActor ! "InitImplicit" - state.finished.await - state.s should be("ReplyImplicit") - } - - "shutdown crashed temporary actor" in { - filterEvents(EventFilter[Exception]("Expected exception")) { - val supervisor = system.actorOf(Props(new Supervisor( - OneForOneStrategy(maxNrOfRetries = 0)(List(classOf[Exception]))))) - val actor = Await.result((supervisor ? Props[CrashingActor]).mapTo[ActorRef], timeout.duration) - actor.isTerminated should be(false) - actor ! "Die" - state.finished.await - Thread.sleep(1.second.dilated.toMillis) - actor.isTerminated should be(true) - system.stop(supervisor) - } - } - } -} diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorMailboxSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorMailboxSpec.scala index 583237ffaf..27c23dd796 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorMailboxSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorMailboxSpec.scala @@ -15,7 +15,7 @@ import java.util.concurrent.TimeUnit import akka.util.Helpers.ConfigOps object ActorMailboxSpec { - val mailboxConf = ConfigFactory.parseString(""" + val mailboxConf = ConfigFactory.parseString(s""" unbounded-dispatcher { mailbox-type = "akka.dispatch.UnboundedMailbox" } @@ -46,7 +46,7 @@ object ActorMailboxSpec { requiring-balancing-bounded-dispatcher { type = "akka.dispatch.BalancingDispatcherConfigurator" - mailbox-requirement = "akka.actor.ActorMailboxSpec$MCBoundedMessageQueueSemantics" + mailbox-requirement = "akka.actor.ActorMailboxSpec$$MCBoundedMessageQueueSemantics" } unbounded-mailbox { @@ -68,7 +68,7 @@ object ActorMailboxSpec { mc-bounded-mailbox { mailbox-capacity = 1000 mailbox-push-timeout-time = 10s - mailbox-type = "akka.actor.ActorMailboxSpec$MCBoundedMailbox" + mailbox-type = "akka.actor.ActorMailboxSpec$$MCBoundedMailbox" } akka.actor.deployment { @@ -142,7 +142,7 @@ object ActorMailboxSpec { } akka.actor.mailbox.requirements { - "akka.actor.ActorMailboxSpec$MCBoundedMessageQueueSemantics" = + "akka.actor.ActorMailboxSpec$$MCBoundedMessageQueueSemantics" = mc-bounded-mailbox } """) diff --git a/akka-actor-tests/src/test/scala/akka/actor/UidClashTest.scala b/akka-actor-tests/src/test/scala/akka/actor/UidClashTest.scala index aa12696d6c..42dc1e9f32 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/UidClashTest.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/UidClashTest.scala @@ -16,12 +16,12 @@ object UidClashTest { @volatile var oldActor: ActorRef = _ - class EvilCollidingActorRef(override val provider: ActorRefProvider, - override val path: ActorPath, - val eventStream: EventStream) extends MinimalActorRef { + private[akka] class EvilCollidingActorRef(override val provider: ActorRefProvider, + override val path: ActorPath, + val eventStream: EventStream) extends MinimalActorRef { //Ignore everything - override def isTerminated(): Boolean = true + override def isTerminated: Boolean = true override def sendSystemMessage(message: SystemMessage): Unit = () override def !(message: Any)(implicit sender: ActorRef = Actor.noSender): Unit = () } 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 fad739a570..104b1425a1 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala @@ -51,6 +51,27 @@ object FutureSpec { final case class Req[T](req: T) final case class Res[T](res: T) + + sealed trait IntAction { def apply(that: Int): Int } + final case class IntAdd(n: Int) extends IntAction { def apply(that: Int) = that + n } + final case class IntSub(n: Int) extends IntAction { def apply(that: Int) = that - n } + final case class IntMul(n: Int) extends IntAction { def apply(that: Int) = that * n } + final case class IntDiv(n: Int) extends IntAction { def apply(that: Int) = that / n } + + sealed trait FutureAction { + def /:(that: Try[Int]): Try[Int] + def /:(that: Future[Int]): Future[Int] + } + + final case class MapAction(action: IntAction)(implicit ec: ExecutionContext) extends FutureAction { + def /:(that: Try[Int]): Try[Int] = that map action.apply + def /:(that: Future[Int]): Future[Int] = that map action.apply + } + + final case class FlatMapAction(action: IntAction)(implicit ec: ExecutionContext) extends FutureAction { + def /:(that: Try[Int]): Try[Int] = that map action.apply + def /:(that: Future[Int]): Future[Int] = that flatMap (n ⇒ Future.successful(action(n))) + } } class JavaFutureSpec extends JavaFutureTests with JUnitSuiteLike @@ -628,7 +649,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa "filter result" in { f { (future, result) ⇒ Await.result((future filter (_ ⇒ true)), timeout.duration) should be(result) - evaluating { Await.result((future filter (_ ⇒ false)), timeout.duration) } should produce[java.util.NoSuchElementException] + intercept[java.util.NoSuchElementException] { Await.result((future filter (_ ⇒ false)), timeout.duration) } } } "transform result with map" in { f((future, result) ⇒ Await.result((future map (_.toString.length)), timeout.duration) should be(result.toString.length)) } @@ -648,7 +669,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa "zip properly" in { f { (future, result) ⇒ Await.result(future zip Promise.successful("foo").future, timeout.duration) should be((result, "foo")) - (evaluating { Await.result(future zip Promise.failed(new RuntimeException("ohnoes")).future, timeout.duration) } should produce[RuntimeException]).getMessage should be("ohnoes") + (intercept[RuntimeException] { Await.result(future zip Promise.failed(new RuntimeException("ohnoes")).future, timeout.duration) }).getMessage should be("ohnoes") } } "not recover from exception" in { f((future, result) ⇒ Await.result(future.recover({ case _ ⇒ "pigdog" }), timeout.duration) should be(result)) } @@ -659,7 +680,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa Await.result(p.future, timeout.duration) should be(result) } } - "not project a failure" in { f((future, result) ⇒ (evaluating { Await.result(future.failed, timeout.duration) } should produce[NoSuchElementException]).getMessage should be("Future.failed not completed with a throwable.")) } + "not project a failure" in { f((future, result) ⇒ (intercept[NoSuchElementException] { Await.result(future.failed, timeout.duration) }).getMessage should be("Future.failed not completed with a throwable.")) } "not perform action on exception" is pending "cast using mapTo" in { f((future, result) ⇒ Await.result(future.mapTo[Boolean].recover({ case _: ClassCastException ⇒ false }), timeout.duration) should be(false)) } } @@ -674,20 +695,20 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa f.getMessage should be(message) }) } - "throw exception with 'get'" in { f((future, message) ⇒ (evaluating { Await.result(future, timeout.duration) } should produce[java.lang.Exception]).getMessage should be(message)) } - "throw exception with 'Await.result'" in { f((future, message) ⇒ (evaluating { Await.result(future, timeout.duration) } should produce[java.lang.Exception]).getMessage should be(message)) } + "throw exception with 'get'" in { f((future, message) ⇒ (intercept[java.lang.Exception] { Await.result(future, timeout.duration) }).getMessage should be(message)) } + "throw exception with 'Await.result'" in { f((future, message) ⇒ (intercept[java.lang.Exception] { Await.result(future, timeout.duration) }).getMessage should be(message)) } "retain exception with filter" in { f { (future, message) ⇒ - (evaluating { Await.result(future filter (_ ⇒ true), timeout.duration) } should produce[java.lang.Exception]).getMessage should be(message) - (evaluating { Await.result(future filter (_ ⇒ false), timeout.duration) } should produce[java.lang.Exception]).getMessage should be(message) + (intercept[java.lang.Exception] { Await.result(future filter (_ ⇒ true), timeout.duration) }).getMessage should be(message) + (intercept[java.lang.Exception] { Await.result(future filter (_ ⇒ false), timeout.duration) }).getMessage should be(message) } } - "retain exception with map" in { f((future, message) ⇒ (evaluating { Await.result(future map (_.toString.length), timeout.duration) } should produce[java.lang.Exception]).getMessage should be(message)) } - "retain exception with flatMap" in { f((future, message) ⇒ (evaluating { Await.result(future flatMap (_ ⇒ Promise.successful[Any]("foo").future), timeout.duration) } should produce[java.lang.Exception]).getMessage should be(message)) } + "retain exception with map" in { f((future, message) ⇒ (intercept[java.lang.Exception] { Await.result(future map (_.toString.length), timeout.duration) }).getMessage should be(message)) } + "retain exception with flatMap" in { f((future, message) ⇒ (intercept[java.lang.Exception] { Await.result(future flatMap (_ ⇒ Promise.successful[Any]("foo").future), timeout.duration) }).getMessage should be(message)) } "not perform action with foreach" is pending "zip properly" in { - f { (future, message) ⇒ (evaluating { Await.result(future zip Promise.successful("foo").future, timeout.duration) } should produce[java.lang.Exception]).getMessage should be(message) } + f { (future, message) ⇒ (intercept[java.lang.Exception] { Await.result(future zip Promise.successful("foo").future, timeout.duration) }).getMessage should be(message) } } "recover from exception" in { f((future, message) ⇒ Await.result(future.recover({ case e if e.getMessage == message ⇒ "pigdog" }), timeout.duration) should be("pigdog")) } "not perform action on result" is pending @@ -702,27 +723,6 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa "always cast successfully using mapTo" in { f((future, message) ⇒ (evaluating { Await.result(future.mapTo[java.lang.Thread], timeout.duration) } should produce[java.lang.Exception]).getMessage should be(message)) } } - sealed trait IntAction { def apply(that: Int): Int } - final case class IntAdd(n: Int) extends IntAction { def apply(that: Int) = that + n } - final case class IntSub(n: Int) extends IntAction { def apply(that: Int) = that - n } - final case class IntMul(n: Int) extends IntAction { def apply(that: Int) = that * n } - final case class IntDiv(n: Int) extends IntAction { def apply(that: Int) = that / n } - - sealed trait FutureAction { - def /:(that: Try[Int]): Try[Int] - def /:(that: Future[Int]): Future[Int] - } - - final case class MapAction(action: IntAction) extends FutureAction { - def /:(that: Try[Int]): Try[Int] = that map action.apply - def /:(that: Future[Int]): Future[Int] = that map action.apply - } - - final case class FlatMapAction(action: IntAction) extends FutureAction { - def /:(that: Try[Int]): Try[Int] = that map action.apply - def /:(that: Future[Int]): Future[Int] = that flatMap (n ⇒ Future.successful(action(n))) - } - implicit def arbFuture: Arbitrary[Future[Int]] = Arbitrary(for (n ← arbitrary[Int]) yield Future(n)) implicit def arbFutureAction: Arbitrary[FutureAction] = Arbitrary { diff --git a/akka-actor-tests/src/test/scala/akka/event/EventStreamSpec.scala b/akka-actor-tests/src/test/scala/akka/event/EventStreamSpec.scala index b884fa7c04..a3ae2bc68d 100644 --- a/akka-actor-tests/src/test/scala/akka/event/EventStreamSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/event/EventStreamSpec.scala @@ -56,9 +56,9 @@ object EventStreamSpec { // class hierarchy for subchannel test class A - class B1 extends A class B2 extends A - class C extends B1 + class B3 extends A + class C extends B2 trait T trait AT extends T @@ -137,12 +137,12 @@ class EventStreamSpec extends AkkaSpec(EventStreamSpec.config) { "manage sub-channels using classes" in { val a = new A - val b1 = new B1 - val b2 = new B2 + val b1 = new B2 + val b2 = new B3 val c = new C val bus = new EventStream(system, false) within(2 seconds) { - bus.subscribe(testActor, classOf[B2]) should be(true) + bus.subscribe(testActor, classOf[B3]) should be(true) bus.publish(c) bus.publish(b2) expectMsg(b2) @@ -151,7 +151,7 @@ class EventStreamSpec extends AkkaSpec(EventStreamSpec.config) { expectMsg(c) bus.publish(b1) expectMsg(b1) - bus.unsubscribe(testActor, classOf[B1]) should be(true) + bus.unsubscribe(testActor, classOf[B2]) should be(true) bus.publish(c) bus.publish(b2) bus.publish(a) diff --git a/akka-actor-tests/src/test/scala/akka/event/LoggerSpec.scala b/akka-actor-tests/src/test/scala/akka/event/LoggerSpec.scala index fed90d381a..229c3da911 100644 --- a/akka-actor-tests/src/test/scala/akka/event/LoggerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/event/LoggerSpec.scala @@ -97,9 +97,10 @@ object LoggerSpec { override def mdc(currentMessage: Any): MDC = { reqId += 1 val always = Map("requestId" -> reqId) + val cmim = "Current Message in MDC" val perMessage = currentMessage match { - case cm @ "Current Message in MDC" ⇒ Map("currentMsg" -> cm, "currentMsgLength" -> cm.length) - case _ ⇒ Map() + case `cmim` ⇒ Map[String, Any]("currentMsg" -> cmim, "currentMsgLength" -> cmim.length) + case _ ⇒ Map() } always ++ perMessage } diff --git a/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala b/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala index 8454c152a4..d14fdf9c4a 100644 --- a/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/serialization/SerializeSpec.scala @@ -20,7 +20,7 @@ import org.apache.commons.codec.binary.Hex.encodeHex object SerializationTests { - val serializeConf = """ + val serializeConf = s""" akka { actor { serialize-messages = off @@ -29,13 +29,13 @@ object SerializationTests { } serialization-bindings { - "akka.serialization.SerializationTests$Person" = java - "akka.serialization.SerializationTests$Address" = java + "akka.serialization.SerializationTests$$Person" = java + "akka.serialization.SerializationTests$$Address" = java "akka.serialization.TestSerializable" = test - "akka.serialization.SerializationTests$PlainMessage" = test - "akka.serialization.SerializationTests$A" = java - "akka.serialization.SerializationTests$B" = test - "akka.serialization.SerializationTests$D" = test + "akka.serialization.SerializationTests$$PlainMessage" = test + "akka.serialization.SerializationTests$$A" = java + "akka.serialization.SerializationTests$$B" = test + "akka.serialization.SerializationTests$$D" = test } } } diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index f0850e4893..81c16e596b 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -294,6 +294,8 @@ object AkkaBuild extends Build { // compile options scalacOptions in Compile ++= Seq("-encoding", "UTF-8", "-target:jvm-1.6", "-feature", "-unchecked", "-Xlog-reflective-calls", "-Xlint"), scalacOptions in Compile ++= (if (deprecation) Seq("-deprecation") else Nil), + scalacOptions in Test := (scalacOptions in Test).value.filterNot(_ == "-Xlog-reflective-calls"), + // -XDignore.symbol.file suppresses sun.misc.Unsafe warnings javacOptions in compile ++= Seq("-encoding", "UTF-8", "-source", "1.6", "-target", "1.6", "-Xlint:unchecked", "-XDignore.symbol.file"), javacOptions in compile ++= (if (deprecation) Seq("-Xlint:deprecation") else Nil), javacOptions in doc ++= Seq("-encoding", "UTF-8", "-source", "1.6"), From 82b8238a9cb99461ab1d0dc53b16c9f3206af3d3 Mon Sep 17 00:00:00 2001 From: Roland Kuhn Date: Fri, 30 Jan 2015 16:34:27 +0100 Subject: [PATCH 3/4] fix warnings in remote and cluster --- .../scala/akka/cluster/ClusterDaemon.scala | 11 +++++----- .../scala/akka/cluster/StressSpec.scala | 20 +++++++++---------- .../ClusterConsistentHashingRouterSpec.scala | 8 ++++---- .../ClusterHeartbeatSenderStateSpec.scala | 6 +++--- .../cluster/HeartbeatNodeRingPerfSpec.scala | 4 ++-- .../akka/cluster/ReachabilityPerfSpec.scala | 20 +++++++++---------- .../akka/remote/testconductor/Player.scala | 7 ++++--- .../PiercingShouldKeepQuarantineSpec.scala | 2 -- .../test/scala/akka/remote/RemotingSpec.scala | 16 +++++++-------- .../SystemMessageDeliveryStressTest.scala | 2 +- 10 files changed, 48 insertions(+), 48 deletions(-) diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterDaemon.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterDaemon.scala index cf50f8a54f..e4cf868a85 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterDaemon.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterDaemon.scala @@ -163,7 +163,7 @@ private[cluster] final class ClusterDaemon(settings: ClusterSettings) extends Ac withDispatcher(context.props.dispatcher), name = "heartbeatReceiver") def receive = { - case msg @ GetClusterCoreRef ⇒ coreSupervisor forward msg + case msg: GetClusterCoreRef.type ⇒ coreSupervisor forward msg case AddOnMemberUpListener(code) ⇒ context.actorOf(Props(classOf[OnMemberUpListener], code).withDeploy(Deploy.local)) case PublisherCreated(publisher) ⇒ @@ -655,10 +655,11 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef) extends Actor with if (statsEnabled) { gossipStats = gossipType match { - case Merge ⇒ gossipStats.incrementMergeCount - case Same ⇒ gossipStats.incrementSameCount - case Newer ⇒ gossipStats.incrementNewerCount - case Older ⇒ gossipStats.incrementOlderCount + case Merge ⇒ gossipStats.incrementMergeCount + case Same ⇒ gossipStats.incrementSameCount + case Newer ⇒ gossipStats.incrementNewerCount + case Older ⇒ gossipStats.incrementOlderCount + case Ignored ⇒ gossipStats // included in receivedGossipCount } } diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/StressSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/StressSpec.scala index cc8d60c8bd..28a7e04a46 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/StressSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/StressSpec.scala @@ -246,21 +246,21 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig { class ClusterResultAggregator(title: String, expectedResults: Int, settings: Settings) extends Actor with ActorLogging { import settings.reportMetricsInterval import settings.infolog - val cluster = Cluster(context.system) - var reportTo: Option[ActorRef] = None - var results = Vector.empty[ClusterResult] - var nodeMetrics = Set.empty[NodeMetrics] - var phiValuesObservedByNode = { + private val cluster = Cluster(context.system) + private var reportTo: Option[ActorRef] = None + private var results = Vector.empty[ClusterResult] + private var nodeMetrics = Set.empty[NodeMetrics] + private var phiValuesObservedByNode = { import akka.cluster.Member.addressOrdering immutable.SortedMap.empty[Address, immutable.SortedSet[PhiValue]] } - var clusterStatsObservedByNode = { + private var clusterStatsObservedByNode = { import akka.cluster.Member.addressOrdering immutable.SortedMap.empty[Address, CurrentInternalStats] } import context.dispatcher - val reportMetricsTask = context.system.scheduler.schedule( + private val reportMetricsTask = context.system.scheduler.schedule( reportMetricsInterval, reportMetricsInterval, self, ReportTick) // subscribe to ClusterMetricsChanged, re-subscribe when restart @@ -441,9 +441,9 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig { } class StatsObserver extends Actor { - val cluster = Cluster(context.system) - var reportTo: Option[ActorRef] = None - var startStats: Option[GossipStats] = None + private val cluster = Cluster(context.system) + private var reportTo: Option[ActorRef] = None + private var startStats: Option[GossipStats] = None override def preStart(): Unit = cluster.subscribe(self, classOf[CurrentInternalStats]) override def postStop(): Unit = cluster.unsubscribe(self) diff --git a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/ClusterConsistentHashingRouterSpec.scala b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/ClusterConsistentHashingRouterSpec.scala index 7d3bddde94..0e0a15828f 100644 --- a/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/ClusterConsistentHashingRouterSpec.scala +++ b/akka-cluster/src/multi-jvm/scala/akka/cluster/routing/ClusterConsistentHashingRouterSpec.scala @@ -37,7 +37,7 @@ object ClusterConsistentHashingRouterMultiJvmSpec extends MultiNodeConfig { val third = role("third") commonConfig(debugConfig(on = false). - withFallback(ConfigFactory.parseString(""" + withFallback(ConfigFactory.parseString(s""" common-router-settings = { router = consistent-hashing-pool nr-of-instances = 10 @@ -48,9 +48,9 @@ object ClusterConsistentHashingRouterMultiJvmSpec extends MultiNodeConfig { } akka.actor.deployment { - /router1 = ${common-router-settings} - /router3 = ${common-router-settings} - /router4 = ${common-router-settings} + /router1 = $${common-router-settings} + /router3 = $${common-router-settings} + /router4 = $${common-router-settings} } """)). withFallback(MultiNodeClusterSpec.clusterConfig)) diff --git a/akka-cluster/src/test/scala/akka/cluster/ClusterHeartbeatSenderStateSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ClusterHeartbeatSenderStateSpec.scala index 184cfea15c..4400182108 100644 --- a/akka-cluster/src/test/scala/akka/cluster/ClusterHeartbeatSenderStateSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/ClusterHeartbeatSenderStateSpec.scala @@ -49,14 +49,14 @@ class ClusterHeartbeatSenderStateSpec extends WordSpec with Matchers { val dd = UniqueAddress(Address("akka.tcp", "sys", "dd", 2552), 4) val ee = UniqueAddress(Address("akka.tcp", "sys", "ee", 2552), 5) - def emptyState: ClusterHeartbeatSenderState = emptyState(aa) + private def emptyState: ClusterHeartbeatSenderState = emptyState(aa) - def emptyState(selfUniqueAddress: UniqueAddress) = ClusterHeartbeatSenderState( + private def emptyState(selfUniqueAddress: UniqueAddress) = ClusterHeartbeatSenderState( ring = HeartbeatNodeRing(selfUniqueAddress, Set(selfUniqueAddress), Set.empty, monitoredByNrOfMembers = 3), oldReceiversNowUnreachable = Set.empty[UniqueAddress], failureDetector = new DefaultFailureDetectorRegistry[Address](() ⇒ new FailureDetectorStub)) - def fd(state: ClusterHeartbeatSenderState, node: UniqueAddress): FailureDetectorStub = + private def fd(state: ClusterHeartbeatSenderState, node: UniqueAddress): FailureDetectorStub = state.failureDetector.asInstanceOf[DefaultFailureDetectorRegistry[Address]].failureDetector(node.address). get.asInstanceOf[FailureDetectorStub] diff --git a/akka-cluster/src/test/scala/akka/cluster/HeartbeatNodeRingPerfSpec.scala b/akka-cluster/src/test/scala/akka/cluster/HeartbeatNodeRingPerfSpec.scala index a4012c2938..5564f3ddfc 100644 --- a/akka-cluster/src/test/scala/akka/cluster/HeartbeatNodeRingPerfSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/HeartbeatNodeRingPerfSpec.scala @@ -23,10 +23,10 @@ class HeartbeatNodeRingPerfSpec extends WordSpec with Matchers { val heartbeatNodeRing = createHeartbeatNodeRingOfSize(nodesSize) - def checkThunkForRing(ring: HeartbeatNodeRing, thunk: HeartbeatNodeRing ⇒ Unit, times: Int): Unit = + private def checkThunkForRing(ring: HeartbeatNodeRing, thunk: HeartbeatNodeRing ⇒ Unit, times: Int): Unit = for (i ← 1 to times) thunk(ring) - def myReceivers(ring: HeartbeatNodeRing): Unit = { + private def myReceivers(ring: HeartbeatNodeRing): Unit = { val r = HeartbeatNodeRing(ring.selfAddress, ring.nodes, Set.empty, ring.monitoredByNrOfMembers) r.myReceivers.isEmpty should be(false) } diff --git a/akka-cluster/src/test/scala/akka/cluster/ReachabilityPerfSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ReachabilityPerfSpec.scala index c79b27a0cb..ae5f6c439e 100644 --- a/akka-cluster/src/test/scala/akka/cluster/ReachabilityPerfSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/ReachabilityPerfSpec.scala @@ -16,7 +16,7 @@ class ReachabilityPerfSpec extends WordSpec with Matchers { val address = Address("akka.tcp", "sys", "a", 2552) val node = Address("akka.tcp", "sys", "a", 2552) - def createReachabilityOfSize(base: Reachability, size: Int): Reachability = + private def createReachabilityOfSize(base: Reachability, size: Int): Reachability = (base /: (1 to size)) { case (r, i) ⇒ val observer = UniqueAddress(address.copy(host = Some("node-" + i)), i) @@ -25,7 +25,7 @@ class ReachabilityPerfSpec extends WordSpec with Matchers { r.unreachable(observer, subject).reachable(observer, subject) } - def addUnreachable(base: Reachability, count: Int): Reachability = { + private def addUnreachable(base: Reachability, count: Int): Reachability = { val observers = base.allObservers.take(count) val subjects = Stream.continually(base.allObservers).flatten.iterator (base /: observers) { @@ -39,43 +39,43 @@ class ReachabilityPerfSpec extends WordSpec with Matchers { val reachability3 = addUnreachable(reachability1, nodesSize / 2) val allowed = reachability1.allObservers - def checkThunkFor(r1: Reachability, r2: Reachability, thunk: (Reachability, Reachability) ⇒ Unit, times: Int): Unit = { + private def checkThunkFor(r1: Reachability, r2: Reachability, thunk: (Reachability, Reachability) ⇒ Unit, times: Int): Unit = { for (i ← 1 to times) { thunk(Reachability(r1.records, r1.versions), Reachability(r2.records, r2.versions)) } } - def checkThunkFor(r1: Reachability, thunk: Reachability ⇒ Unit, times: Int): Unit = { + private def checkThunkFor(r1: Reachability, thunk: Reachability ⇒ Unit, times: Int): Unit = { for (i ← 1 to times) { thunk(Reachability(r1.records, r1.versions)) } } - def merge(expectedRecords: Int)(r1: Reachability, r2: Reachability): Unit = { + private def merge(expectedRecords: Int)(r1: Reachability, r2: Reachability): Unit = { r1.merge(allowed, r2).records.size should be(expectedRecords) } - def checkStatus(r1: Reachability): Unit = { + private def checkStatus(r1: Reachability): Unit = { val record = r1.records.head r1.status(record.observer, record.subject) should be(record.status) } - def checkAggregatedStatus(r1: Reachability): Unit = { + private def checkAggregatedStatus(r1: Reachability): Unit = { val record = r1.records.head r1.status(record.subject) should be(record.status) } - def allUnreachableOrTerminated(r1: Reachability): Unit = { + private def allUnreachableOrTerminated(r1: Reachability): Unit = { val record = r1.records.head r1.allUnreachableOrTerminated.isEmpty should be(false) } - def allUnreachable(r1: Reachability): Unit = { + private def allUnreachable(r1: Reachability): Unit = { val record = r1.records.head r1.allUnreachable.isEmpty should be(false) } - def recordsFrom(r1: Reachability): Unit = { + private def recordsFrom(r1: Reachability): Unit = { r1.allObservers.foreach { o ⇒ r1.recordsFrom(o) should not be be(null) } diff --git a/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Player.scala b/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Player.scala index 6e81c91372..29d5864d4a 100644 --- a/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Player.scala +++ b/akka-multi-node-testkit/src/main/scala/akka/remote/testconductor/Player.scala @@ -193,10 +193,11 @@ private[akka] class ClientFSM(name: RoleName, controllerAddr: InetSocketAddress) case Event(ToServer(msg), d @ Data(Some(channel), None)) ⇒ channel.write(msg) val token = msg match { - case EnterBarrier(barrier, timeout) ⇒ barrier - case GetAddress(node) ⇒ node.name + case EnterBarrier(barrier, timeout) ⇒ Some(barrier -> sender()) + case GetAddress(node) ⇒ Some(node.name -> sender()) + case _ ⇒ None } - stay using d.copy(runningOp = Some(token -> sender())) + stay using d.copy(runningOp = token) case Event(ToServer(op), Data(channel, Some((token, _)))) ⇒ log.error("cannot write {} while waiting for {}", op, token) stay diff --git a/akka-remote-tests/src/multi-jvm/scala/akka/remote/PiercingShouldKeepQuarantineSpec.scala b/akka-remote-tests/src/multi-jvm/scala/akka/remote/PiercingShouldKeepQuarantineSpec.scala index 2da78d76cc..03db880499 100644 --- a/akka-remote-tests/src/multi-jvm/scala/akka/remote/PiercingShouldKeepQuarantineSpec.scala +++ b/akka-remote-tests/src/multi-jvm/scala/akka/remote/PiercingShouldKeepQuarantineSpec.scala @@ -1,11 +1,9 @@ package akka.remote -import language.postfixOps import scala.concurrent.duration._ import com.typesafe.config.ConfigFactory import akka.actor._ import akka.testkit._ -import akka.remote.AddressUidExtension import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec } import akka.remote.testconductor.RoleName diff --git a/akka-remote/src/test/scala/akka/remote/RemotingSpec.scala b/akka-remote/src/test/scala/akka/remote/RemotingSpec.scala index 6100c64f8d..adb2c117ab 100644 --- a/akka-remote/src/test/scala/akka/remote/RemotingSpec.scala +++ b/akka-remote/src/test/scala/akka/remote/RemotingSpec.scala @@ -586,16 +586,16 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D """).withFallback(config) val otherSelection = thisSystem.actorSelection(s"akka.tcp://other-system@localhost:${otherAddress.getPort}/user/echo") otherSelection.tell("ping", probeSender) - probe.expectNoMsg(1 seconds) + probe.expectNoMsg(1.seconds) val otherSystem = ActorSystem("other-system", otherConfig) try { muteSystem(otherSystem) - probe.expectNoMsg(2 seconds) + probe.expectNoMsg(2.seconds) otherSystem.actorOf(Props[Echo2], "echo") - within(5 seconds) { + within(5.seconds) { awaitAssert { otherSelection.tell("ping", probeSender) - assert(probe.expectMsgType[(String, ActorRef)](500 millis)._1 == "pong") + assert(probe.expectMsgType[(String, ActorRef)](500.millis)._1 == "pong") } } } finally { @@ -624,18 +624,18 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D """).withFallback(config) val otherSelection = thisSystem.actorSelection(s"akka.tcp://other-system@localhost:${otherAddress.getPort}/user/echo") otherSelection.tell("ping", thisSender) - thisProbe.expectNoMsg(1 seconds) + thisProbe.expectNoMsg(1.seconds) val otherSystem = ActorSystem("other-system", otherConfig) try { muteSystem(otherSystem) - thisProbe.expectNoMsg(2 seconds) + thisProbe.expectNoMsg(2.seconds) val otherProbe = new TestProbe(otherSystem) val otherSender = otherProbe.ref val thisSelection = otherSystem.actorSelection(s"akka.tcp://this-system@localhost:${port(thisSystem, "tcp")}/user/echo") - within(5 seconds) { + within(5.seconds) { awaitAssert { thisSelection.tell("ping", otherSender) - assert(otherProbe.expectMsgType[(String, ActorRef)](500 millis)._1 == "pong") + assert(otherProbe.expectMsgType[(String, ActorRef)](500.millis)._1 == "pong") } } } finally { diff --git a/akka-remote/src/test/scala/akka/remote/transport/SystemMessageDeliveryStressTest.scala b/akka-remote/src/test/scala/akka/remote/transport/SystemMessageDeliveryStressTest.scala index 021281c183..50109d58f9 100644 --- a/akka-remote/src/test/scala/akka/remote/transport/SystemMessageDeliveryStressTest.scala +++ b/akka-remote/src/test/scala/akka/remote/transport/SystemMessageDeliveryStressTest.scala @@ -63,7 +63,7 @@ object SystemMessageDeliveryStressTest { } """) - class SystemMessageSequenceVerifier(system: ActorSystem, testActor: ActorRef) extends MinimalActorRef { + private[akka] class SystemMessageSequenceVerifier(system: ActorSystem, testActor: ActorRef) extends MinimalActorRef { val provider = RARP(system).provider val path = provider.tempPath() From a029a90502f44c80f9daf8720fd8028239a1841f Mon Sep 17 00:00:00 2001 From: Roland Kuhn Date: Fri, 30 Jan 2015 18:34:03 +0100 Subject: [PATCH 4/4] fix warnings in contrib, docs, osgi, persistence and slf4j --- .../pattern/DistributedPubSubMediator.scala | 2 +- .../contrib/pattern/ReceivePipeline.scala | 2 +- .../contrib/pattern/ReceivePipelineSpec.scala | 82 +++++++++++-------- .../rst/additional/code/docs/faq/Faq.scala | 3 + .../java/code/docs/future/FutureDocTest.java | 1 + .../scala/code/docs/actor/ActorDocSpec.scala | 4 +- .../scala/code/docs/actor/FSMDocSpec.scala | 23 ++++-- .../docs/actor/FaultHandlingDocSample.scala | 7 +- .../docs/actor/FaultHandlingDocSpec.scala | 2 +- .../code/docs/actor/TypedActorDocSpec.scala | 4 +- .../rst/scala/code/docs/io/EchoServer.scala | 7 +- .../rst/scala/code/docs/io/UdpDocSpec.scala | 5 +- .../docs/persistence/PersistenceDocSpec.scala | 43 +++++----- .../scala/akka/osgi/DefaultOSGiLogger.scala | 2 +- .../scala/akka/persistence/SnapshotSpec.scala | 4 +- .../akka/event/slf4j/Slf4jLoggerSpec.scala | 8 +- project/AkkaBuild.scala | 6 +- 17 files changed, 116 insertions(+), 89 deletions(-) diff --git a/akka-contrib/src/main/scala/akka/contrib/pattern/DistributedPubSubMediator.scala b/akka-contrib/src/main/scala/akka/contrib/pattern/DistributedPubSubMediator.scala index 5db3783d79..b128944ff1 100644 --- a/akka-contrib/src/main/scala/akka/contrib/pattern/DistributedPubSubMediator.scala +++ b/akka-contrib/src/main/scala/akka/contrib/pattern/DistributedPubSubMediator.scala @@ -191,7 +191,7 @@ object DistributedPubSubMediator { def business: Receive - def receive = business orElse defaultReceive + def receive = business.orElse[Any, Unit](defaultReceive) def remove(ref: ActorRef): Unit = { if (subscribers.contains(ref)) diff --git a/akka-contrib/src/main/scala/akka/contrib/pattern/ReceivePipeline.scala b/akka-contrib/src/main/scala/akka/contrib/pattern/ReceivePipeline.scala index 600049c5c9..94f397b55e 100644 --- a/akka-contrib/src/main/scala/akka/contrib/pattern/ReceivePipeline.scala +++ b/akka-contrib/src/main/scala/akka/contrib/pattern/ReceivePipeline.scala @@ -38,7 +38,7 @@ trait ReceivePipeline extends Actor { val around = aroundCache match { case Some((`receive`, cached)) ⇒ cached case _ ⇒ - val zipped = pipeline.foldRight(receive)((outer, inner) ⇒ outer(inner) orElse inner) + val zipped = pipeline.foldRight[Receive](receive)((outer, inner) ⇒ outer(inner).orElse[Any, Unit](inner)) aroundCache = Some((receive, zipped)) zipped } diff --git a/akka-contrib/src/test/scala/akka/contrib/pattern/ReceivePipelineSpec.scala b/akka-contrib/src/test/scala/akka/contrib/pattern/ReceivePipelineSpec.scala index 90239675a1..597e260004 100644 --- a/akka-contrib/src/test/scala/akka/contrib/pattern/ReceivePipelineSpec.scala +++ b/akka-contrib/src/test/scala/akka/contrib/pattern/ReceivePipelineSpec.scala @@ -2,49 +2,60 @@ package akka.contrib.pattern import akka.testkit.{ ImplicitSender, AkkaSpec } import akka.actor.{ Actor, Props } +import scala.concurrent.duration._ -class ReplierActor extends Actor with ReceivePipeline { - def receive: Actor.Receive = becomeAndReply - def becomeAndReply: Actor.Receive = { - case "become" ⇒ context.become(justReply) - case m ⇒ sender ! m +object ReceivePipelineSpec { + + class ReplierActor extends Actor with ReceivePipeline { + def receive: Actor.Receive = becomeAndReply + def becomeAndReply: Actor.Receive = { + case "become" ⇒ context.become(justReply) + case m ⇒ sender ! m + } + def justReply: Actor.Receive = { + case m ⇒ sender ! m + } } - def justReply: Actor.Receive = { - case m ⇒ sender ! m + + case class IntList(l: List[Int]) { + override def toString: String = s"IntList(${l.mkString(", ")})" } -} -trait ListBuilderInterceptor { - this: ReceivePipeline ⇒ + trait ListBuilderInterceptor { + this: ReceivePipeline ⇒ - pipelineOuter(inner ⇒ - { - case n: Int ⇒ inner((n until n + 3).toList) - }) -} + pipelineOuter(inner ⇒ + { + case n: Int ⇒ inner(IntList((n until n + 3).toList)) + }) + } -trait AdderInterceptor { - this: ReceivePipeline ⇒ + trait AdderInterceptor { + this: ReceivePipeline ⇒ - pipelineInner(inner ⇒ - { - case n: Int ⇒ inner(n + 10) - case l: List[Int] ⇒ inner(l.map(_ + 10)) - case "explicitly ignored" ⇒ - }) -} + pipelineInner(inner ⇒ + { + case n: Int ⇒ inner(n + 10) + case IntList(l) ⇒ inner(IntList(l.map(_ + 10))) + case "explicitly ignored" ⇒ + }) + } -trait ToStringInterceptor { - this: ReceivePipeline ⇒ + trait ToStringInterceptor { + this: ReceivePipeline ⇒ + + pipelineInner(inner ⇒ + { + case i: Int ⇒ inner(i.toString) + case IntList(l) ⇒ inner(l.toString) + case other: Iterable[_] ⇒ inner(other.toString) + }) + } - pipelineInner(inner ⇒ - { - case i: Int ⇒ inner(i.toString) - case l: Iterable[_] ⇒ inner(l.toString()) - }) } class ReceivePipelineSpec extends AkkaSpec with ImplicitSender { + import ReceivePipelineSpec._ "A ReceivePipeline" must { @@ -82,7 +93,8 @@ class ReceivePipelineSpec extends AkkaSpec with ImplicitSender { val replier = system.actorOf(Props( new ReplierActor with ListBuilderInterceptor with AdderInterceptor with ToStringInterceptor)) replier ! "explicitly ignored" - expectNoMsg() + replier ! 8L // unhandled by all interceptors but still replied + expectMsg(8L) } "support changing behavior without losing the interceptions" in { @@ -101,9 +113,9 @@ class ReceivePipelineSpec extends AkkaSpec with ImplicitSender { val innerOuterReplier = system.actorOf(Props( new ReplierActor with AdderInterceptor with ListBuilderInterceptor)) outerInnerReplier ! 4 - expectMsg(List(14, 15, 16)) + expectMsg(IntList(List(14, 15, 16))) innerOuterReplier ! 6 - expectMsg(List(16, 17, 18)) + expectMsg(IntList(List(16, 17, 18))) } } @@ -231,7 +243,7 @@ object MixinSample extends App { // The Dude says 'Yeah, well, you know, that's just, like, your opinion, man.' //#mixin-actor - system.shutdown() + system.terminate() } object UnhandledSample extends App { diff --git a/akka-docs/rst/additional/code/docs/faq/Faq.scala b/akka-docs/rst/additional/code/docs/faq/Faq.scala index 6d270fdd97..9ba5025db1 100644 --- a/akka-docs/rst/additional/code/docs/faq/Faq.scala +++ b/akka-docs/rst/additional/code/docs/faq/Faq.scala @@ -21,6 +21,9 @@ class MyActor extends Actor { case BarMessage(bar) => sender() ! BazMessage("Got " + bar) // warning here: // "match may not be exhaustive. It would fail on the following input: FooMessage(_)" + //#exhaustiveness-check + case FooMessage(_) => // avoid the warning in our build logs + //#exhaustiveness-check } } } diff --git a/akka-docs/rst/java/code/docs/future/FutureDocTest.java b/akka-docs/rst/java/code/docs/future/FutureDocTest.java index d947c49645..4279728760 100644 --- a/akka-docs/rst/java/code/docs/future/FutureDocTest.java +++ b/akka-docs/rst/java/code/docs/future/FutureDocTest.java @@ -554,6 +554,7 @@ public class FutureDocTest { } @Test(expected = IllegalStateException.class) + @SuppressWarnings("unchecked") public void useAfter() throws Exception { //#after final ExecutionContext ec = system.dispatcher(); diff --git a/akka-docs/rst/scala/code/docs/actor/ActorDocSpec.scala b/akka-docs/rst/scala/code/docs/actor/ActorDocSpec.scala index d3f296c332..42dd5cb7f4 100644 --- a/akka-docs/rst/scala/code/docs/actor/ActorDocSpec.scala +++ b/akka-docs/rst/scala/code/docs/actor/ActorDocSpec.scala @@ -92,7 +92,7 @@ class ActorWithMessagesWrapper { import MyActor._ def receive = { case Greeting(greeter) => log.info(s"I was greeted by $greeter.") - case Goodbye => log.info("Someone said goodbye to me.") + case Goodbye => log.info("Someone said goodbye to me.") } } //#messages-in-companion @@ -229,7 +229,7 @@ class Consumer extends Actor with ActorLogging with ConsumerBehavior { class ProducerConsumer extends Actor with ActorLogging with ProducerBehavior with ConsumerBehavior { - def receive = producerBehavior orElse consumerBehavior + def receive = producerBehavior.orElse[Any, Unit](consumerBehavior) } // protocol diff --git a/akka-docs/rst/scala/code/docs/actor/FSMDocSpec.scala b/akka-docs/rst/scala/code/docs/actor/FSMDocSpec.scala index 41cae10068..f8ad23af39 100644 --- a/akka-docs/rst/scala/code/docs/actor/FSMDocSpec.scala +++ b/akka-docs/rst/scala/code/docs/actor/FSMDocSpec.scala @@ -12,13 +12,10 @@ import akka.util.ByteString import akka.actor.Props import scala.collection.immutable -class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit { - - //#fsm-code-elided - //#simple-imports - import akka.actor.{ ActorRef, FSM } - import scala.concurrent.duration._ - //#simple-imports +object FSMDocSpec { + // messages and data types + //#test-code + import akka.actor.ActorRef //#simple-events // received events final case class SetTarget(ref: ActorRef) @@ -38,6 +35,17 @@ class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit { case object Uninitialized extends Data final case class Todo(target: ActorRef, queue: immutable.Seq[Any]) extends Data //#simple-state + //#test-code +} + +class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit { + import FSMDocSpec._ + + //#fsm-code-elided + //#simple-imports + import akka.actor.{ ActorRef, FSM } + import scala.concurrent.duration._ + //#simple-imports //#simple-fsm class Buncher extends FSM[State, Data] { @@ -56,6 +64,7 @@ class FSMDocSpec extends MyFavoriteTestFrameWorkPlusAkkaTestKit { case Active -> Idle => stateData match { case Todo(ref, queue) => ref ! Batch(queue) + case _ => // nothing to do } } //#transition-elided diff --git a/akka-docs/rst/scala/code/docs/actor/FaultHandlingDocSample.scala b/akka-docs/rst/scala/code/docs/actor/FaultHandlingDocSample.scala index 3e9454647e..d26b689c5a 100644 --- a/akka-docs/rst/scala/code/docs/actor/FaultHandlingDocSample.scala +++ b/akka-docs/rst/scala/code/docs/actor/FaultHandlingDocSample.scala @@ -113,7 +113,8 @@ class Worker extends Actor with ActorLogging { //#messages object CounterService { final case class Increment(n: Int) - case object GetCurrentCount + sealed abstract class GetCurrentCount + case object GetCurrentCount extends GetCurrentCount final case class CurrentCount(key: String, count: Long) class ServiceUnavailable(msg: String) extends RuntimeException(msg) @@ -176,9 +177,9 @@ class CounterService extends Actor { for ((replyTo, msg) <- backlog) c.tell(msg, sender = replyTo) backlog = IndexedSeq.empty - case msg @ Increment(n) => forwardOrPlaceInBacklog(msg) + case msg: Increment => forwardOrPlaceInBacklog(msg) - case msg @ GetCurrentCount => forwardOrPlaceInBacklog(msg) + case msg: GetCurrentCount => forwardOrPlaceInBacklog(msg) case Terminated(actorRef) if Some(actorRef) == storage => // After 3 restarts the storage child is stopped. diff --git a/akka-docs/rst/scala/code/docs/actor/FaultHandlingDocSpec.scala b/akka-docs/rst/scala/code/docs/actor/FaultHandlingDocSpec.scala index f8025e9854..92f46ad1fc 100644 --- a/akka-docs/rst/scala/code/docs/actor/FaultHandlingDocSpec.scala +++ b/akka-docs/rst/scala/code/docs/actor/FaultHandlingDocSpec.scala @@ -108,7 +108,7 @@ class FaultHandlingDocSpec extends TestKit(ActorSystem("FaultHandlingDocSpec", t } "A supervisor" must "apply the chosen strategy for its child" in { - //#testkit + //#testkit //#create val supervisor = system.actorOf(Props[Supervisor], "supervisor") diff --git a/akka-docs/rst/scala/code/docs/actor/TypedActorDocSpec.scala b/akka-docs/rst/scala/code/docs/actor/TypedActorDocSpec.scala index 6d6fc0227e..4c107e8254 100644 --- a/akka-docs/rst/scala/code/docs/actor/TypedActorDocSpec.scala +++ b/akka-docs/rst/scala/code/docs/actor/TypedActorDocSpec.scala @@ -143,7 +143,7 @@ class TypedActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) { //#typed-actor-call-strict //#typed-actor-calls - Await.result(fSquare, 3 seconds) should be(100) + Await.result(fSquare, 3.seconds) should be(100) oSquare should be(Some(100)) @@ -193,7 +193,7 @@ class TypedActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) { TypedActor(system).poisonPill(awesomeFooBar) //#typed-actor-supercharge-usage - Await.result(f, 3 seconds) should be("YES") + Await.result(f, 3.seconds) should be("YES") } "typed router pattern" in { diff --git a/akka-docs/rst/scala/code/docs/io/EchoServer.scala b/akka-docs/rst/scala/code/docs/io/EchoServer.scala index 0b515a709c..4f1fea6976 100644 --- a/akka-docs/rst/scala/code/docs/io/EchoServer.scala +++ b/akka-docs/rst/scala/code/docs/io/EchoServer.scala @@ -70,18 +70,19 @@ class EchoManager(handlerClass: Class[_]) extends Actor with ActorLogging { } +//#echo-handler object EchoHandler { + final case class Ack(offset: Int) extends Tcp.Event + def props(connection: ActorRef, remote: InetSocketAddress): Props = Props(classOf[EchoHandler], connection, remote) } -//#echo-handler class EchoHandler(connection: ActorRef, remote: InetSocketAddress) extends Actor with ActorLogging { import Tcp._ - - final case class Ack(offset: Int) extends Event + import EchoHandler._ // sign death pact: this actor terminates when connection breaks context watch connection diff --git a/akka-docs/rst/scala/code/docs/io/UdpDocSpec.scala b/akka-docs/rst/scala/code/docs/io/UdpDocSpec.scala index be37d04e47..5a03629050 100644 --- a/akka-docs/rst/scala/code/docs/io/UdpDocSpec.scala +++ b/akka-docs/rst/scala/code/docs/io/UdpDocSpec.scala @@ -87,8 +87,9 @@ object ScalaUdpDocSpec { //#connected case msg: String => connection ! UdpConnected.Send(ByteString(msg)) - case d @ UdpConnected.Disconnect => connection ! d - case UdpConnected.Disconnected => context.stop(self) + case UdpConnected.Disconnect => + connection ! UdpConnected.Disconnect + case UdpConnected.Disconnected => context.stop(self) } } //#connected diff --git a/akka-docs/rst/scala/code/docs/persistence/PersistenceDocSpec.scala b/akka-docs/rst/scala/code/docs/persistence/PersistenceDocSpec.scala index 87b32f9664..3fb0a3154f 100644 --- a/akka-docs/rst/scala/code/docs/persistence/PersistenceDocSpec.scala +++ b/akka-docs/rst/scala/code/docs/persistence/PersistenceDocSpec.scala @@ -4,13 +4,19 @@ package docs.persistence -import akka.actor.{ Actor, ActorSystem, Props } +import akka.actor.{ Actor, ActorRef, ActorSystem, Props } import akka.persistence._ import com.typesafe.config.ConfigFactory import scala.concurrent.duration._ import scala.language.postfixOps -trait PersistenceDocSpec { + +object PersistenceDocSpec { + + trait SomeOtherMessage + + val persistentActor: ActorRef = ??? + val config = """ //#auto-update-interval @@ -21,13 +27,7 @@ trait PersistenceDocSpec { //#auto-update """ - trait SomeOtherMessage - - implicit val system: ActorSystem - - import system._ - - new AnyRef { + object Recovery { trait MyPersistentActor1 extends PersistentActor { //#recover-on-start-disabled override def preStart() = () @@ -45,7 +45,6 @@ trait PersistenceDocSpec { //#recover-on-start-custom } - val persistentActor = system.deadLetters //#recover-explicit persistentActor ! Recover() //#recover-explicit @@ -69,7 +68,7 @@ trait PersistenceDocSpec { } } - new AnyRef { + object NoRecovery { trait MyPersistentActor1 extends PersistentActor { //#recover-fully-disabled override def preStart() = self ! Recover(toSequenceNr = 0L) @@ -77,7 +76,7 @@ trait PersistenceDocSpec { } } - new AnyRef { + object PersistenceId { trait PersistentActorMethods { //#persistence-id def persistenceId: String @@ -101,7 +100,7 @@ trait PersistenceDocSpec { } } - new AnyRef { + object AtLeastOnce { //#at-least-once-example import akka.actor.{ Actor, ActorPath } import akka.persistence.AtLeastOnceDelivery @@ -145,7 +144,7 @@ trait PersistenceDocSpec { //#at-least-once-example } - new AnyRef { + object SaveSnapshot { class MyPersistentActor extends PersistentActor { override def persistenceId = "my-stable-persistence-id" @@ -164,7 +163,7 @@ trait PersistenceDocSpec { } } - new AnyRef { + object OfferSnapshot { class MyPersistentActor extends PersistentActor { override def persistenceId = "my-stable-persistence-id" @@ -183,8 +182,6 @@ trait PersistenceDocSpec { import akka.actor.Props - val persistentActor = system.actorOf(Props[MyPersistentActor]) - //#snapshot-criteria persistentActor ! Recover(fromSnapshot = SnapshotSelectionCriteria( maxSequenceNr = 457L, @@ -192,9 +189,7 @@ trait PersistenceDocSpec { //#snapshot-criteria } - new AnyRef { - - val persistentActor = system.actorOf(Props[MyPersistentActor]()) + object PersistAsync { //#persist-async class MyPersistentActor extends PersistentActor { @@ -228,9 +223,8 @@ trait PersistenceDocSpec { //#persist-async } - new AnyRef { - val persistentActor = system.actorOf(Props[MyPersistentActor]()) + object Defer { //#defer class MyPersistentActor extends PersistentActor { @@ -268,9 +262,12 @@ trait PersistenceDocSpec { //#defer-caller } - new AnyRef { + + object View { import akka.actor.Props + val system: ActorSystem = ??? + //#view class MyView extends PersistentView { override def persistenceId: String = "some-persistence-id" diff --git a/akka-osgi/src/main/scala/akka/osgi/DefaultOSGiLogger.scala b/akka-osgi/src/main/scala/akka/osgi/DefaultOSGiLogger.scala index d2af5f5042..028f3042fc 100644 --- a/akka-osgi/src/main/scala/akka/osgi/DefaultOSGiLogger.scala +++ b/akka-osgi/src/main/scala/akka/osgi/DefaultOSGiLogger.scala @@ -17,7 +17,7 @@ class DefaultOSGiLogger extends DefaultLogger { val messageFormat = " %s | %s | %s | %s" - override def receive: Receive = uninitialisedReceive orElse super.receive + override def receive: Receive = uninitialisedReceive.orElse[Any, Unit](super.receive) /** * Behaviour of the logger that waits for its LogService diff --git a/akka-persistence/src/test/scala/akka/persistence/SnapshotSpec.scala b/akka-persistence/src/test/scala/akka/persistence/SnapshotSpec.scala index dfca6a7b76..7f6c096b98 100644 --- a/akka-persistence/src/test/scala/akka/persistence/SnapshotSpec.scala +++ b/akka-persistence/src/test/scala/akka/persistence/SnapshotSpec.scala @@ -14,8 +14,8 @@ object SnapshotSpec { var state = List.empty[String] override def receiveRecover: Receive = { - case payload: String ⇒ state = s"${payload}-${lastSequenceNr}" :: state - case SnapshotOffer(_, snapshot: List[String]) ⇒ state = snapshot + case payload: String ⇒ state = s"${payload}-${lastSequenceNr}" :: state + case SnapshotOffer(_, snapshot: List[_]) ⇒ state = snapshot.asInstanceOf[List[String]] } override def receiveCommand = { diff --git a/akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jLoggerSpec.scala b/akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jLoggerSpec.scala index 5db4f3745c..e88b862253 100644 --- a/akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jLoggerSpec.scala +++ b/akka-slf4j/src/test/scala/akka/event/slf4j/Slf4jLoggerSpec.scala @@ -26,6 +26,8 @@ object Slf4jLoggerSpec { } """ + case class StringWithMDC(s: String, mdc: Map[String, Any]) + class LogProducer extends Actor with DiagnosticActorLogging { def receive = { @@ -33,7 +35,7 @@ object Slf4jLoggerSpec { log.error(e, e.getMessage) case (s: String, x: Int, y: Int) ⇒ log.info(s, x, y) - case (s: String, mdc: Map[String, Any]) ⇒ + case StringWithMDC(s, mdc) ⇒ log.mdc(mdc) log.info(s) log.clearMDC() @@ -96,7 +98,7 @@ class Slf4jLoggerSpec extends AkkaSpec(Slf4jLoggerSpec.config) with BeforeAndAft } "put custom MDC values when specified" in { - producer ! ("Message with custom MDC values", Map("ticketNumber" -> 3671, "ticketDesc" -> "Custom MDC Values")) + producer ! StringWithMDC("Message with custom MDC values", Map("ticketNumber" -> 3671, "ticketDesc" -> "Custom MDC Values")) awaitCond(outputString.contains("----"), 5 seconds) val s = outputString @@ -109,7 +111,7 @@ class Slf4jLoggerSpec extends AkkaSpec(Slf4jLoggerSpec.config) with BeforeAndAft } "Support null values in custom MDC" in { - producer ! ("Message with null custom MDC values", Map("ticketNumber" -> 3671, "ticketDesc" -> null)) + producer ! StringWithMDC("Message with null custom MDC values", Map("ticketNumber" -> 3671, "ticketDesc" -> null)) awaitCond(outputString.contains("----"), 5 seconds) val s = outputString diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index 81c16e596b..d1b56ec30b 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -287,17 +287,17 @@ object AkkaBuild extends Build { pomIncludeRepository := (_ => false) // do not leak internal repositories during staging ) - private def deprecation: Boolean = System.getProperty("akka.deprecation", "false").toBoolean + private def allWarnings: Boolean = System.getProperty("akka.allwarnings", "false").toBoolean lazy val defaultSettings = baseSettings ++ resolverSettings ++ TestExtras.Filter.settings ++ Protobuf.settings ++ Seq( // compile options scalacOptions in Compile ++= Seq("-encoding", "UTF-8", "-target:jvm-1.6", "-feature", "-unchecked", "-Xlog-reflective-calls", "-Xlint"), - scalacOptions in Compile ++= (if (deprecation) Seq("-deprecation") else Nil), + scalacOptions in Compile ++= (if (allWarnings) Seq("-deprecation") else Nil), scalacOptions in Test := (scalacOptions in Test).value.filterNot(_ == "-Xlog-reflective-calls"), // -XDignore.symbol.file suppresses sun.misc.Unsafe warnings javacOptions in compile ++= Seq("-encoding", "UTF-8", "-source", "1.6", "-target", "1.6", "-Xlint:unchecked", "-XDignore.symbol.file"), - javacOptions in compile ++= (if (deprecation) Seq("-Xlint:deprecation") else Nil), + javacOptions in compile ++= (if (allWarnings) Seq("-Xlint:deprecation") else Nil), javacOptions in doc ++= Seq("-encoding", "UTF-8", "-source", "1.6"), incOptions := incOptions.value.withNameHashing(true),