[not for merge] Warnings as errors in akka-actor (#26634)

Warnings as errors in akka-actor
This commit is contained in:
Arnout Engelen 2019-04-03 13:13:44 +02:00 committed by GitHub
parent 18802420fb
commit 76479b8770
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
68 changed files with 313 additions and 100 deletions

View file

@ -0,0 +1,36 @@
/*
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.util
import akka.util.ByteIterator.ByteArrayIterator
import org.scalatest._
class ByteIteratorSpec extends WordSpec with Matchers {
"A ByteIterator" should {
"correctly implement indexOf" in {
// Since the 'indexOf' operator invalidates the iterator,
// we must create a new one for each test:
def freshIterator(): ByteIterator = ByteArrayIterator(Array(0x20, 0x20, 0x10, 0x20, 0x20, 0x10))
freshIterator().indexOf(0x20) should be(0)
freshIterator().indexOf(0x10) should be(2)
freshIterator().indexOf(0x20, 1) should be(1)
freshIterator().indexOf(0x10, 1) should be(2)
freshIterator().indexOf(0x10, 3) should be(5)
// There is also an indexOf with another signature, which is hard to invoke :D
def otherIndexOf(iterator: ByteIterator, byte: Byte, from: Int): Int =
classOf[ByteIterator]
.getMethod("indexOf", classOf[Byte], classOf[Int])
.invoke(iterator, byte.asInstanceOf[Object], from.asInstanceOf[Object])
.asInstanceOf[Int]
otherIndexOf(freshIterator(), 0x20, 1) should be(1)
otherIndexOf(freshIterator(), 0x10, 1) should be(2)
otherIndexOf(freshIterator(), 0x10, 3) should be(5)
}
}
}

View file

@ -16,9 +16,6 @@ import akka.dispatch.forkjoin.ThreadLocalRandom
@InternalApi @InternalApi
sealed private[akka] trait RoutingLogic[T] { sealed private[akka] trait RoutingLogic[T] {
/**
* @param routees available routees, will contain at least one element. Must not be mutated by select logic.
*/
def selectRoutee(): ActorRef[T] def selectRoutee(): ActorRef[T]
/** /**

View file

@ -4,10 +4,13 @@
package akka.japi.function package akka.japi.function
import com.github.ghik.silencer.silent
[3..22#/** [3..22#/**
* A Function interface. Used to create 1-arg first-class-functions is Java. * A Function interface. Used to create 1-arg first-class-functions is Java.
* `Serializable` is needed to be able to grab line number for Java 8 lambdas. * `Serializable` is needed to be able to grab line number for Java 8 lambdas.
*/ */
@silent
@SerialVersionUID(##1L) @SerialVersionUID(##1L)
trait Function1[[#-T1#], +R] extends java.io.Serializable { trait Function1[[#-T1#], +R] extends java.io.Serializable {
@throws(classOf[Exception]) @throws(classOf[Exception])
@ -21,6 +24,7 @@ trait Function1[[#-T1#], +R] extends java.io.Serializable {
* A Procedure is like a Function, but it doesn't produce a return value. * A Procedure is like a Function, but it doesn't produce a return value.
* `Serializable` is needed to be able to grab line number for Java 8 lambdas. * `Serializable` is needed to be able to grab line number for Java 8 lambdas.
*/ */
@silent
@SerialVersionUID(##1L) @SerialVersionUID(##1L)
trait Procedure1[[#-T1#]] extends java.io.Serializable { trait Procedure1[[#-T1#]] extends java.io.Serializable {
@throws(classOf[Exception]) @throws(classOf[Exception])

View file

@ -5,9 +5,11 @@
package akka.compat package akka.compat
import akka.annotation.InternalApi import akka.annotation.InternalApi
import scala.concurrent.{ ExecutionContext, Future => SFuture } import scala.concurrent.{ ExecutionContext, Future => SFuture }
import scala.collection.immutable import scala.collection.immutable
import akka.util.ccompat._ import akka.util.ccompat._
import com.github.ghik.silencer.silent
/** /**
* INTERNAL API * INTERNAL API
@ -17,7 +19,7 @@ import akka.util.ccompat._
* *
* Remove these classes as soon as support for Scala 2.11 is dropped! * Remove these classes as soon as support for Scala 2.11 is dropped!
*/ */
@InternalApi private[akka] object Future { @silent @InternalApi private[akka] object Future {
def fold[T, R](futures: IterableOnce[SFuture[T]])(zero: R)(op: (R, T) => R)( def fold[T, R](futures: IterableOnce[SFuture[T]])(zero: R)(op: (R, T) => R)(
implicit executor: ExecutionContext): SFuture[R] = implicit executor: ExecutionContext): SFuture[R] =
SFuture.fold[T, R](futures)(zero)(op)(executor) SFuture.fold[T, R](futures)(zero)(op)(executor)

View file

@ -277,8 +277,8 @@ object ByteIterator {
final override def takeWhile(p: Byte => Boolean): this.type = { final override def takeWhile(p: Byte => Boolean): this.type = {
var stop = false var stop = false
var builder = new ListBuffer[ByteArrayIterator] val builder = new ListBuffer[ByteArrayIterator]
while (!stop && !iterators.isEmpty) { while (!stop && iterators.nonEmpty) {
val lastLen = current.len val lastLen = current.len
current.takeWhile(p) current.takeWhile(p)
if (current.hasNext) builder += current if (current.hasNext) builder += current
@ -460,7 +460,11 @@ abstract class ByteIterator extends BufferedIterator[Byte] {
override def indexWhere(p: Byte => Boolean): Int = indexWhere(p, 0) override def indexWhere(p: Byte => Boolean): Int = indexWhere(p, 0)
override def indexWhere(p: Byte => Boolean, from: Int): Int = { override def indexWhere(p: Byte => Boolean, from: Int): Int = {
var index = from var index = 0
while (index < from) {
next()
index += 1
}
var found = false var found = false
while (!found && hasNext) if (p(next())) { while (!found && hasNext) if (p(next())) {
found = true found = true
@ -471,7 +475,7 @@ abstract class ByteIterator extends BufferedIterator[Byte] {
} }
def indexOf(elem: Byte): Int = indexOf(elem, 0) def indexOf(elem: Byte): Int = indexOf(elem, 0)
def indexOf(elem: Byte, from: Int): Int = indexWhere(_ == elem, 0) def indexOf(elem: Byte, from: Int): Int = indexWhere(_ == elem, from)
override def indexOf[B >: Byte](elem: B): Int = indexOf[B](elem, 0) override def indexOf[B >: Byte](elem: B): Int = indexOf[B](elem, 0)
override def indexOf[B >: Byte](elem: B, from: Int): Int = indexWhere(_ == elem, from) override def indexOf[B >: Byte](elem: B, from: Int): Int = indexWhere(_ == elem, from)

View file

@ -9,10 +9,10 @@ import akka.util.Collections.EmptyImmutableSeq
import java.nio.{ ByteBuffer, ByteOrder } import java.nio.{ ByteBuffer, ByteOrder }
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.collection.BufferedIterator
import scala.collection.LinearSeq import scala.collection.LinearSeq
import scala.collection.mutable.ListBuffer import scala.collection.mutable.ListBuffer
import scala.reflect.ClassTag import scala.reflect.ClassTag
import akka.util.ccompat._
object ByteIterator { object ByteIterator {
object ByteArrayIterator { object ByteArrayIterator {
@ -280,7 +280,7 @@ object ByteIterator {
final override def takeWhile(p: Byte => Boolean): this.type = { final override def takeWhile(p: Byte => Boolean): this.type = {
var stop = false var stop = false
var builder = new ListBuffer[ByteArrayIterator] val builder = new ListBuffer[ByteArrayIterator]
while (!stop && !iterators.isEmpty) { while (!stop && !iterators.isEmpty) {
val lastLen = current.len val lastLen = current.len
current.takeWhile(p) current.takeWhile(p)
@ -415,8 +415,11 @@ abstract class ByteIterator extends BufferedIterator[Byte] {
protected def clear(): Unit protected def clear(): Unit
def ++(that: IterableOnce[Byte]): ByteIterator = def ++(that: IterableOnce[Byte]): ByteIterator = {
if (that.isEmpty) this else ByteIterator.ByteArrayIterator(that.toArray) val it = that.iterator
if (it.isEmpty) this
else ByteIterator.ByteArrayIterator(it.toArray)
}
// *must* be overridden by derived classes. This construction is necessary // *must* be overridden by derived classes. This construction is necessary
// to specialize the return type, as the method is already implemented in // to specialize the return type, as the method is already implemented in

View file

@ -4,8 +4,6 @@
package akka.util package akka.util
import akka.compat._
import java.io.{ ObjectInputStream, ObjectOutputStream } import java.io.{ ObjectInputStream, ObjectOutputStream }
import java.nio.{ ByteBuffer, ByteOrder } import java.nio.{ ByteBuffer, ByteOrder }
import java.lang.{ Iterable => JIterable } import java.lang.{ Iterable => JIterable }
@ -14,10 +12,8 @@ import scala.annotation.{ tailrec, varargs }
import scala.collection.mutable.{ Builder, WrappedArray } import scala.collection.mutable.{ Builder, WrappedArray }
import scala.collection.{ immutable, mutable } import scala.collection.{ immutable, mutable }
import scala.collection.immutable.{ IndexedSeq, IndexedSeqOps, StrictOptimizedSeqOps, VectorBuilder } import scala.collection.immutable.{ IndexedSeq, IndexedSeqOps, StrictOptimizedSeqOps, VectorBuilder }
import scala.collection.generic.CanBuildFrom
import scala.reflect.ClassTag import scala.reflect.ClassTag
import java.nio.charset.{ Charset, StandardCharsets } import java.nio.charset.{ Charset, StandardCharsets }
import akka.util.ccompat._
object ByteString { object ByteString {
@ -214,7 +210,6 @@ object ByteString {
if (n <= 0) this if (n <= 0) this
else toByteString1.drop(n) else toByteString1.drop(n)
override def indexOf[B >: Byte](elem: B): Int = indexOf(elem, 0)
override def indexOf[B >: Byte](elem: B, from: Int): Int = { override def indexOf[B >: Byte](elem: B, from: Int): Int = {
if (from >= length) -1 if (from >= length) -1
else { else {
@ -373,7 +368,6 @@ object ByteString {
} }
} }
override def indexOf[B >: Byte](elem: B): Int = indexOf(elem, 0)
override def indexOf[B >: Byte](elem: B, from: Int): Int = { override def indexOf[B >: Byte](elem: B, from: Int): Int = {
if (from >= length) -1 if (from >= length) -1
else { else {
@ -392,7 +386,7 @@ object ByteString {
private[akka] object ByteStrings extends Companion { private[akka] object ByteStrings extends Companion {
def apply(bytestrings: Vector[ByteString1]): ByteString = def apply(bytestrings: Vector[ByteString1]): ByteString =
new ByteStrings(bytestrings, (0 /: bytestrings)(_ + _.length)) new ByteStrings(bytestrings, bytestrings.foldLeft(0)(_ + _.length))
def apply(bytestrings: Vector[ByteString1], length: Int): ByteString = new ByteStrings(bytestrings, length) def apply(bytestrings: Vector[ByteString1], length: Int): ByteString = new ByteStrings(bytestrings, length)
@ -474,7 +468,7 @@ object ByteString {
/** Avoid `iterator` in performance sensitive code, call ops directly on ByteString instead */ /** Avoid `iterator` in performance sensitive code, call ops directly on ByteString instead */
override def iterator: ByteIterator.MultiByteArrayIterator = override def iterator: ByteIterator.MultiByteArrayIterator =
ByteIterator.MultiByteArrayIterator(bytestrings.toStream.map { _.iterator }) ByteIterator.MultiByteArrayIterator(bytestrings.to(LazyList).map { _.iterator })
def ++(that: ByteString): ByteString = { def ++(that: ByteString): ByteString = {
if (that.isEmpty) this if (that.isEmpty) this
@ -602,7 +596,6 @@ object ByteString {
new ByteStrings(bytestrings(fullDrops).drop1(remainingToDrop) +: bytestrings.drop(fullDrops + 1), length - n) new ByteStrings(bytestrings(fullDrops).drop1(remainingToDrop) +: bytestrings.drop(fullDrops + 1), length - n)
} }
override def indexOf[B >: Byte](elem: B): Int = indexOf(elem, 0)
override def indexOf[B >: Byte](elem: B, from: Int): Int = { override def indexOf[B >: Byte](elem: B, from: Int): Int = {
if (from >= length) -1 if (from >= length) -1
else { else {
@ -635,12 +628,12 @@ object ByteString {
@SerialVersionUID(1L) @SerialVersionUID(1L)
private class SerializationProxy(@transient private var orig: ByteString) extends Serializable { private class SerializationProxy(@transient private var orig: ByteString) extends Serializable {
private def writeObject(out: ObjectOutputStream) { private def writeObject(out: ObjectOutputStream): Unit = {
out.writeByte(orig.byteStringCompanion.SerializationIdentity) out.writeByte(orig.byteStringCompanion.SerializationIdentity)
orig.writeToOutputStream(out) orig.writeToOutputStream(out)
} }
private def readObject(in: ObjectInputStream) { private def readObject(in: ObjectInputStream): Unit = {
val serializationId = in.readByte() val serializationId = in.readByte()
orig = Companion(from = serializationId).readFromInputStream(in) orig = Companion(from = serializationId).readFromInputStream(in)
@ -731,10 +724,10 @@ sealed abstract class ByteString
override def splitAt(n: Int): (ByteString, ByteString) = (take(n), drop(n)) override def splitAt(n: Int): (ByteString, ByteString) = (take(n), drop(n))
override def indexWhere(p: Byte => Boolean): Int = iterator.indexWhere(p) override def indexWhere(p: Byte => Boolean, from: Int): Int = iterator.indexWhere(p, from)
// optimized in subclasses // optimized in subclasses
override def indexOf[B >: Byte](elem: B): Int = indexOf(elem, 0) override def indexOf[B >: Byte](elem: B, from: Int): Int = indexOf(elem, from)
override def grouped(size: Int): Iterator[ByteString] = { override def grouped(size: Int): Iterator[ByteString] = {
if (size <= 0) { if (size <= 0) {
@ -784,9 +777,7 @@ sealed abstract class ByteString
* @param buffer a ByteBuffer to copy bytes to * @param buffer a ByteBuffer to copy bytes to
* @return the number of bytes actually copied * @return the number of bytes actually copied
*/ */
// *must* be overridden by derived classes. def copyToBuffer(@unused buffer: ByteBuffer): Int
def copyToBuffer(buffer: ByteBuffer): Int =
throw new UnsupportedOperationException("Method copyToBuffer is not implemented in ByteString")
/** /**
* Create a new ByteString with all contents compacted into a single, * Create a new ByteString with all contents compacted into a single,
@ -879,8 +870,11 @@ object CompactByteString {
/** /**
* Creates a new CompactByteString by traversing bytes. * Creates a new CompactByteString by traversing bytes.
*/ */
def apply(bytes: IterableOnce[Byte]): CompactByteString = def apply(bytes: IterableOnce[Byte]): CompactByteString = {
if (bytes.isEmpty) empty else ByteString.ByteString1C(bytes.toArray) val it = bytes.iterator
if (it.isEmpty) empty
else ByteString.ByteString1C(it.toArray)
}
/** /**
* Creates a new CompactByteString by converting from integral numbers to bytes. * Creates a new CompactByteString by converting from integral numbers to bytes.
@ -1022,7 +1016,7 @@ final class ByteStringBuilder extends Builder[Byte, ByteString] {
override def addAll(xs: IterableOnce[Byte]): this.type = { override def addAll(xs: IterableOnce[Byte]): this.type = {
xs match { xs match {
case _ if xs.isEmpty => case _ if xs.iterator.isEmpty =>
// do nothing // do nothing
case b: ByteString1C => case b: ByteString1C =>
clearTemp() clearTemp()

View file

@ -20,7 +20,7 @@ import scala.collection.immutable
def fold[T, R](futures: IterableOnce[SFuture[T]])(zero: R)(op: (R, T) => R)( def fold[T, R](futures: IterableOnce[SFuture[T]])(zero: R)(op: (R, T) => R)(
implicit executor: ExecutionContext): SFuture[R] = { implicit executor: ExecutionContext): SFuture[R] = {
// This will have performance implications since the elements are copied to a Vector // This will have performance implications since the elements are copied to a Vector
SFuture.foldLeft[T, R](futures.to(immutable.Iterable))(zero)(op)(executor) SFuture.foldLeft[T, R](futures.iterator.to(immutable.Iterable))(zero)(op)(executor)
} }
def fold[T, R](futures: immutable.Iterable[SFuture[T]])(zero: R)(op: (R, T) => R)( def fold[T, R](futures: immutable.Iterable[SFuture[T]])(zero: R)(op: (R, T) => R)(
@ -30,7 +30,7 @@ import scala.collection.immutable
def reduce[T, R >: T](futures: IterableOnce[SFuture[T]])(op: (R, T) => R)( def reduce[T, R >: T](futures: IterableOnce[SFuture[T]])(op: (R, T) => R)(
implicit executor: ExecutionContext): SFuture[R] = { implicit executor: ExecutionContext): SFuture[R] = {
// This will have performance implications since the elements are copied to a Vector // This will have performance implications since the elements are copied to a Vector
SFuture.reduceLeft[T, R](futures.to(immutable.Iterable))(op)(executor) SFuture.reduceLeft[T, R](futures.iterator.to(immutable.Iterable))(op)(executor)
} }
def reduce[T, R >: T](futures: immutable.Iterable[SFuture[T]])(op: (R, T) => R)( def reduce[T, R >: T](futures: immutable.Iterable[SFuture[T]])(op: (R, T) => R)(
@ -40,7 +40,7 @@ import scala.collection.immutable
def find[T](futures: IterableOnce[SFuture[T]])(p: T => Boolean)( def find[T](futures: IterableOnce[SFuture[T]])(p: T => Boolean)(
implicit executor: ExecutionContext): SFuture[Option[T]] = { implicit executor: ExecutionContext): SFuture[Option[T]] = {
// This will have performance implications since the elements are copied to a Vector // This will have performance implications since the elements are copied to a Vector
SFuture.find[T](futures.to(immutable.Iterable))(p)(executor) SFuture.find[T](futures.iterator.to(immutable.Iterable))(p)(executor)
} }
def find[T](futures: immutable.Iterable[SFuture[T]])(p: T => Boolean)( def find[T](futures: immutable.Iterable[SFuture[T]])(p: T => Boolean)(

View file

@ -11,6 +11,7 @@ import scala.runtime.BoxedUnit
import java.util.Optional import java.util.Optional
import akka.util.JavaDurationConverters import akka.util.JavaDurationConverters
import com.github.ghik.silencer.silent
import scala.concurrent.ExecutionContextExecutor import scala.concurrent.ExecutionContextExecutor
import scala.concurrent.duration.Duration import scala.concurrent.duration.Duration
@ -279,6 +280,7 @@ abstract class AbstractActor extends Actor {
// TODO In 2.6.0 we can remove deprecation and make the method final // TODO In 2.6.0 we can remove deprecation and make the method final
@deprecated("Override preRestart with message parameter with Optional type instead", "2.5.0") @deprecated("Override preRestart with message parameter with Optional type instead", "2.5.0")
@throws(classOf[Exception]) @throws(classOf[Exception])
@silent
override def preRestart(reason: Throwable, message: Option[Any]): Unit = { override def preRestart(reason: Throwable, message: Option[Any]): Unit = {
import scala.compat.java8.OptionConverters._ import scala.compat.java8.OptionConverters._
preRestart(reason, message.asJava) preRestart(reason, message.asJava)

View file

@ -19,6 +19,7 @@ import akka.event.Logging.{ Debug, Error, LogEvent }
import akka.japi.Procedure import akka.japi.Procedure
import akka.util.{ unused, Reflect } import akka.util.{ unused, Reflect }
import akka.annotation.InternalApi import akka.annotation.InternalApi
import com.github.ghik.silencer.silent
/** /**
* The actor context - the view of the actor cell from the actor. * The actor context - the view of the actor cell from the actor.
@ -442,6 +443,7 @@ private[akka] object ActorCell {
* supported APIs in this place. This is not the API you were looking * supported APIs in this place. This is not the API you were looking
* for! (waves hand) * for! (waves hand)
*/ */
@silent
private[akka] class ActorCell( private[akka] class ActorCell(
val system: ActorSystemImpl, val system: ActorSystemImpl,
val self: InternalActorRef, val self: InternalActorRef,

View file

@ -6,9 +6,11 @@ package akka.actor
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.pattern.ask import akka.pattern.ask
import scala.concurrent.Await import scala.concurrent.Await
import akka.util.Helpers.ConfigOps import akka.util.Helpers.ConfigOps
import akka.util.JavaDurationConverters._ import akka.util.JavaDurationConverters._
import com.github.ghik.silencer.silent
/** /**
* This object contains elements which make writing actors and related code * This object contains elements which make writing actors and related code
@ -164,5 +166,6 @@ object Inbox {
/** /**
* Create a new Inbox within the given system. * Create a new Inbox within the given system.
*/ */
@silent
def create(system: ActorSystem): Inbox = ActorDSL.inbox()(system) def create(system: ActorSystem): Inbox = ActorDSL.inbox()(system)
} }

View file

@ -9,6 +9,8 @@ import akka.japi.Util.immutableSeq
import java.net.MalformedURLException import java.net.MalformedURLException
import java.lang.{ StringBuilder => JStringBuilder } import java.lang.{ StringBuilder => JStringBuilder }
import com.github.ghik.silencer.silent
/** /**
* Java API * Java API
*/ */
@ -149,6 +151,7 @@ object ActorPath {
* references are compared the unique id of the actor is not taken into account * references are compared the unique id of the actor is not taken into account
* when comparing actor paths. * when comparing actor paths.
*/ */
@silent
@SerialVersionUID(1L) @SerialVersionUID(1L)
sealed trait ActorPath extends Comparable[ActorPath] with Serializable { sealed trait ActorPath extends Comparable[ActorPath] with Serializable {
@ -274,13 +277,13 @@ final case class RootActorPath(address: Address, name: String = "/") extends Act
override def elements: immutable.Iterable[String] = ActorPath.emptyActorPath override def elements: immutable.Iterable[String] = ActorPath.emptyActorPath
override val toString: String = address + name override val toString: String = address.toString + name
override val toSerializationFormat: String = toString override val toSerializationFormat: String = toString
override def toStringWithAddress(addr: Address): String = override def toStringWithAddress(addr: Address): String =
if (address.host.isDefined) address + name if (address.host.isDefined) address.toString + name
else addr + name else addr.toString + name
override def toSerializationFormatWithAddress(addr: Address): String = toStringWithAddress(addr) override def toSerializationFormatWithAddress(addr: Address): String = toStringWithAddress(addr)

View file

@ -532,7 +532,7 @@ private[akka] class LocalActorRefProvider private[akka] (
override def stop(): Unit = { override def stop(): Unit = {
causeOfTermination.trySuccess( causeOfTermination.trySuccess(
Terminated(provider.rootGuardian)(existenceConfirmed = true, addressTerminated = true)) //Idempotent Terminated(provider.rootGuardian)(existenceConfirmed = true, addressTerminated = true)) //Idempotent
terminationPromise.tryCompleteWith(causeOfTermination.future) // Signal termination downstream, idempotent terminationPromise.completeWith(causeOfTermination.future) // Signal termination downstream, idempotent
} }
@deprecated("Use context.watch(actor) and receive Terminated(actor)", "2.2") @deprecated("Use context.watch(actor) and receive Terminated(actor)", "2.2")

View file

@ -23,12 +23,14 @@ import akka.dispatch.ExecutionContexts
import scala.compat.java8.FutureConverters import scala.compat.java8.FutureConverters
import akka.util.ccompat._ import akka.util.ccompat._
import com.github.ghik.silencer.silent
/** /**
* An ActorSelection is a logical view of a section of an ActorSystem's tree of Actors, * An ActorSelection is a logical view of a section of an ActorSystem's tree of Actors,
* allowing for broadcasting of messages to that section. * allowing for broadcasting of messages to that section.
*/ */
@SerialVersionUID(1L) @SerialVersionUID(1L)
@ccompatUsedUntil213
abstract class ActorSelection extends Serializable { abstract class ActorSelection extends Serializable {
this: ScalaActorSelection => this: ScalaActorSelection =>
@ -324,6 +326,7 @@ private[akka] final case class ActorSelectionMessage(
/** /**
* INTERNAL API * INTERNAL API
*/ */
@silent
@SerialVersionUID(1L) @SerialVersionUID(1L)
private[akka] sealed trait SelectionPathElement private[akka] sealed trait SelectionPathElement

View file

@ -9,7 +9,7 @@ import java.util.concurrent._
import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicReference
import com.typesafe.config.{ Config, ConfigFactory } import com.typesafe.config.{ Config, ConfigFactory }
import akka.event.{ LoggingFilterWithMarker, _ } import akka.event._
import akka.dispatch._ import akka.dispatch._
import akka.japi.Util.immutableSeq import akka.japi.Util.immutableSeq
import akka.actor.dungeon.ChildrenContainer import akka.actor.dungeon.ChildrenContainer
@ -1081,7 +1081,7 @@ private[akka] class ActorSystemImpl(
(toDie.toSeq.sorted.mkString("\n" + indent + " | toDie: ", "\n" + indent + " | ", "")) (toDie.toSeq.sorted.mkString("\n" + indent + " | toDie: ", "\n" + indent + " | ", ""))
case x @ (ChildrenContainer.TerminatedChildrenContainer | ChildrenContainer.EmptyChildrenContainer) => case x @ (ChildrenContainer.TerminatedChildrenContainer | ChildrenContainer.EmptyChildrenContainer) =>
x.toString x.toString
case n: ChildrenContainer.NormalChildrenContainer => n.c.size + " children" case n: ChildrenContainer.NormalChildrenContainer => n.c.size.toString + " children"
case x => Logging.simpleName(x) case x => Logging.simpleName(x)
}) + }) +
(if (cell.childrenRefs.children.isEmpty) "" else "\n") + (if (cell.childrenRefs.children.isEmpty) "" else "\n") +

View file

@ -8,6 +8,7 @@ import java.util.concurrent.atomic.AtomicReference
import akka.routing._ import akka.routing._
import akka.util.WildcardIndex import akka.util.WildcardIndex
import com.github.ghik.silencer.silent
import com.typesafe.config._ import com.typesafe.config._
import scala.annotation.tailrec import scala.annotation.tailrec
@ -92,6 +93,7 @@ trait Scope {
def withFallback(other: Scope): Scope def withFallback(other: Scope): Scope
} }
@silent
@SerialVersionUID(1L) @SerialVersionUID(1L)
abstract class LocalScope extends Scope abstract class LocalScope extends Scope
@ -100,6 +102,7 @@ abstract class LocalScope extends Scope
* which do not set a different scope. It is also the only scope handled by * which do not set a different scope. It is also the only scope handled by
* the LocalActorRefProvider. * the LocalActorRefProvider.
*/ */
@silent
@SerialVersionUID(1L) @SerialVersionUID(1L)
case object LocalScope extends LocalScope { case object LocalScope extends LocalScope {
@ -114,6 +117,7 @@ case object LocalScope extends LocalScope {
/** /**
* This is the default value and as such allows overrides. * This is the default value and as such allows overrides.
*/ */
@silent
@SerialVersionUID(1L) @SerialVersionUID(1L)
abstract class NoScopeGiven extends Scope abstract class NoScopeGiven extends Scope
@SerialVersionUID(1L) @SerialVersionUID(1L)

View file

@ -9,10 +9,10 @@ import scala.concurrent.duration.Duration
import scala.collection.mutable import scala.collection.mutable
import scala.concurrent.duration.FiniteDuration import scala.concurrent.duration.FiniteDuration
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.routing.{ Deafen, Listen, Listeners } import akka.routing.{ Deafen, Listen, Listeners }
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.util.{ unused, JavaDurationConverters } import akka.util.{ unused, JavaDurationConverters }
import com.github.ghik.silencer.silent
object FSM { object FSM {
@ -225,6 +225,7 @@ object FSM {
* Modify state transition descriptor with new state data. The data will be * Modify state transition descriptor with new state data. The data will be
* set when transitioning to the new state. * set when transitioning to the new state.
*/ */
@silent
def using(@deprecatedName('nextStateDate) nextStateData: D): State[S, D] = { def using(@deprecatedName('nextStateDate) nextStateData: D): State[S, D] = {
copy(stateData = nextStateData) copy(stateData = nextStateData)
} }

View file

@ -33,6 +33,7 @@ private[akka] case object ChildNameReserved extends ChildStats
* ChildRestartStats is the statistics kept by every parent Actor for every child Actor * ChildRestartStats is the statistics kept by every parent Actor for every child Actor
* and is used for SupervisorStrategies to know how to deal with problems that occur for the children. * and is used for SupervisorStrategies to know how to deal with problems that occur for the children.
*/ */
@ccompatUsedUntil213
final case class ChildRestartStats( final case class ChildRestartStats(
child: ActorRef, child: ActorRef,
var maxNrOfRetriesCount: Int = 0, var maxNrOfRetriesCount: Int = 0,

View file

@ -6,6 +6,8 @@ package akka.actor
import akka.japi.Creator import akka.japi.Creator
import akka.util.Reflect import akka.util.Reflect
import com.github.ghik.silencer.silent
import scala.collection.immutable import scala.collection.immutable
/** /**
@ -40,6 +42,7 @@ private[akka] object IndirectActorProducer {
val CreatorConsumerClass = classOf[CreatorConsumer] val CreatorConsumerClass = classOf[CreatorConsumer]
val TypedCreatorFunctionConsumerClass = classOf[TypedCreatorFunctionConsumer] val TypedCreatorFunctionConsumerClass = classOf[TypedCreatorFunctionConsumer]
@silent
def apply(clazz: Class[_], args: immutable.Seq[Any]): IndirectActorProducer = { def apply(clazz: Class[_], args: immutable.Seq[Any]): IndirectActorProducer = {
if (classOf[IndirectActorProducer].isAssignableFrom(clazz)) { if (classOf[IndirectActorProducer].isAssignableFrom(clazz)) {
def get1stArg[T]: T = args.head.asInstanceOf[T] def get1stArg[T]: T = args.head.asInstanceOf[T]

View file

@ -21,7 +21,7 @@ class ReflectiveDynamicAccess(val classLoader: ClassLoader) extends DynamicAcces
Try[Class[_ <: T]]({ Try[Class[_ <: T]]({
val c = Class.forName(fqcn, false, classLoader).asInstanceOf[Class[_ <: T]] val c = Class.forName(fqcn, false, classLoader).asInstanceOf[Class[_ <: T]]
val t = implicitly[ClassTag[T]].runtimeClass val t = implicitly[ClassTag[T]].runtimeClass
if (t.isAssignableFrom(c)) c else throw new ClassCastException(t + " is not assignable from " + c) if (t.isAssignableFrom(c)) c else throw new ClassCastException(t.toString + " is not assignable from " + c)
}) })
override def createInstanceFor[T: ClassTag](clazz: Class[_], args: immutable.Seq[(Class[_], AnyRef)]): Try[T] = override def createInstanceFor[T: ClassTag](clazz: Class[_], args: immutable.Seq[(Class[_], AnyRef)]): Try[T] =

View file

@ -14,6 +14,7 @@ import akka.event.Logging.Warning
import akka.util.{ unused, Unsafe } import akka.util.{ unused, Unsafe }
import akka.dispatch._ import akka.dispatch._
import akka.dispatch.sysmsg._ import akka.dispatch.sysmsg._
import com.github.ghik.silencer.silent
import scala.util.control.NonFatal import scala.util.control.NonFatal
@ -47,8 +48,8 @@ private[akka] class RepointableActorRef(
* processing the very first message (i.e. before Cell.start()). Hence there * processing the very first message (i.e. before Cell.start()). Hence there
* are two refs here, one for each function, and they are switched just so. * are two refs here, one for each function, and they are switched just so.
*/ */
@volatile private var _cellDoNotCallMeDirectly: Cell = _ @silent @volatile private var _cellDoNotCallMeDirectly: Cell = _
@volatile private var _lookupDoNotCallMeDirectly: Cell = _ @silent @volatile private var _lookupDoNotCallMeDirectly: Cell = _
def underlying: Cell = Unsafe.instance.getObjectVolatile(this, cellOffset).asInstanceOf[Cell] def underlying: Cell = Unsafe.instance.getObjectVolatile(this, cellOffset).asInstanceOf[Cell]
def lookup = Unsafe.instance.getObjectVolatile(this, lookupOffset).asInstanceOf[Cell] def lookup = Unsafe.instance.getObjectVolatile(this, lookupOffset).asInstanceOf[Cell]

View file

@ -5,6 +5,8 @@
package akka.actor package akka.actor
import akka.util.JavaDurationConverters import akka.util.JavaDurationConverters
import com.github.ghik.silencer.silent
import scala.concurrent.ExecutionContext import scala.concurrent.ExecutionContext
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
@ -47,6 +49,7 @@ trait Scheduler {
* *
* Java & Scala API * Java & Scala API
*/ */
@silent
final def schedule(initialDelay: FiniteDuration, interval: FiniteDuration, receiver: ActorRef, message: Any)( final def schedule(initialDelay: FiniteDuration, interval: FiniteDuration, receiver: ActorRef, message: Any)(
implicit implicit
executor: ExecutionContext, executor: ExecutionContext,

View file

@ -6,10 +6,12 @@ package akka.actor.dsl
import scala.concurrent.Await import scala.concurrent.Await
import akka.actor.ActorLogging import akka.actor.ActorLogging
import scala.collection.immutable.TreeSet import scala.collection.immutable.TreeSet
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor.Cancellable import akka.actor.Cancellable
import akka.actor.Actor import akka.actor.Actor
import scala.collection.mutable.Queue import scala.collection.mutable.Queue
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.actor.ActorRef import akka.actor.ActorRef
@ -17,9 +19,11 @@ import akka.util.Timeout
import akka.actor.Status import akka.actor.Status
import java.util.concurrent.TimeoutException import java.util.concurrent.TimeoutException
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import akka.pattern.ask import akka.pattern.ask
import akka.actor.ActorDSL import akka.actor.ActorDSL
import akka.actor.Props import akka.actor.Props
import com.github.ghik.silencer.silent
/** /**
* INTERNAL API * INTERNAL API
@ -43,6 +47,7 @@ private[akka] object Inbox {
} }
@silent
trait Inbox { this: ActorDSL.type => trait Inbox { this: ActorDSL.type =>
import Inbox._ import Inbox._

View file

@ -9,10 +9,10 @@ import java.util.Optional
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.util.control.NonFatal import scala.util.control.NonFatal
import scala.collection.immutable import scala.collection.immutable
import akka.actor._ import akka.actor._
import akka.serialization.{ Serialization, SerializationExtension, Serializers } import akka.serialization.{ Serialization, SerializationExtension, Serializers }
import akka.util.{ Helpers, Unsafe } import akka.util.{ Helpers, Unsafe }
import com.github.ghik.silencer.silent
private[akka] object Children { private[akka] object Children {
val GetNobody = () => Nobody val GetNobody = () => Nobody
@ -22,6 +22,7 @@ private[akka] trait Children { this: ActorCell =>
import ChildrenContainer._ import ChildrenContainer._
@silent
@volatile @volatile
private var _childrenRefsDoNotCallMeDirectly: ChildrenContainer = EmptyChildrenContainer private var _childrenRefsDoNotCallMeDirectly: ChildrenContainer = EmptyChildrenContainer
@ -48,7 +49,7 @@ private[akka] trait Children { this: ActorCell =>
private[akka] def attachChild(props: Props, name: String, systemService: Boolean): ActorRef = private[akka] def attachChild(props: Props, name: String, systemService: Boolean): ActorRef =
makeChild(this, props, checkName(name), async = true, systemService = systemService) makeChild(this, props, checkName(name), async = true, systemService = systemService)
@volatile private var _functionRefsDoNotCallMeDirectly = Map.empty[String, FunctionRef] @silent @volatile private var _functionRefsDoNotCallMeDirectly = Map.empty[String, FunctionRef]
private def functionRefs: Map[String, FunctionRef] = private def functionRefs: Map[String, FunctionRef] =
Unsafe.instance.getObjectVolatile(this, AbstractActorCell.functionRefsOffset).asInstanceOf[Map[String, FunctionRef]] Unsafe.instance.getObjectVolatile(this, AbstractActorCell.functionRefsOffset).asInstanceOf[Map[String, FunctionRef]]
@ -101,7 +102,7 @@ private[akka] trait Children { this: ActorCell =>
refs.valuesIterator.foreach(_.stop()) refs.valuesIterator.foreach(_.stop())
} }
@volatile private var _nextNameDoNotCallMeDirectly = 0L @silent @volatile private var _nextNameDoNotCallMeDirectly = 0L
final protected def randomName(sb: java.lang.StringBuilder): String = { final protected def randomName(sb: java.lang.StringBuilder): String = {
val num = Unsafe.instance.getAndAddLong(this, AbstractActorCell.nextNameOffset, 1) val num = Unsafe.instance.getAndAddLong(this, AbstractActorCell.nextNameOffset, 1)
Helpers.base64(num, sb) Helpers.base64(num, sb)

View file

@ -140,7 +140,7 @@ private[akka] object ChildrenContainer {
} }
override def toString = override def toString =
if (c.size > 20) c.size + " children" if (c.size > 20) c.size.toString + " children"
else c.mkString("children:\n ", "\n ", "") else c.mkString("children:\n ", "\n ", "")
} }
@ -208,7 +208,7 @@ private[akka] object ChildrenContainer {
override def isNormal: Boolean = reason == UserRequest override def isNormal: Boolean = reason == UserRequest
override def toString = override def toString =
if (c.size > 20) c.size + " children" if (c.size > 20) c.size.toString + " children"
else c.mkString("children (" + toDie.size + " terminating):\n ", "\n ", "\n") + toDie else c.mkString("children (" + toDie.size + " terminating):\n ", "\n ", "\n") + toDie
} }

View file

@ -9,6 +9,7 @@ import akka.event.Logging.{ Debug, Warning }
import akka.actor.{ Actor, ActorCell, ActorRef, ActorRefScope, Address, InternalActorRef, MinimalActorRef, Terminated } import akka.actor.{ Actor, ActorCell, ActorRef, ActorRefScope, Address, InternalActorRef, MinimalActorRef, Terminated }
import akka.event.AddressTerminatedTopic import akka.event.AddressTerminatedTopic
import akka.util.unused import akka.util.unused
import com.github.ghik.silencer.silent
private[akka] trait DeathWatch { this: ActorCell => private[akka] trait DeathWatch { this: ActorCell =>
@ -112,6 +113,7 @@ private[akka] trait DeathWatch { this: ActorCell =>
// TODO this should be removed and be replaced with `set - subject` // TODO this should be removed and be replaced with `set - subject`
// when all actor references have uid, i.e. actorFor is removed // when all actor references have uid, i.e. actorFor is removed
@silent
private def removeFromMap[T](subject: ActorRef, map: Map[ActorRef, T]): Map[ActorRef, T] = private def removeFromMap[T](subject: ActorRef, map: Map[ActorRef, T]): Map[ActorRef, T] =
if (subject.path.uid != ActorCell.undefinedUid) (map - subject) - new UndefinedUidActorRef(subject) if (subject.path.uid != ActorCell.undefinedUid) (map - subject) - new UndefinedUidActorRef(subject)
else map.filterKeys(_.path != subject.path).toMap else map.filterKeys(_.path != subject.path).toMap

View file

@ -5,7 +5,6 @@
package akka.actor.dungeon package akka.actor.dungeon
import scala.annotation.tailrec import scala.annotation.tailrec
import akka.AkkaException import akka.AkkaException
import akka.dispatch.{ Envelope, Mailbox } import akka.dispatch.{ Envelope, Mailbox }
import akka.dispatch.sysmsg._ import akka.dispatch.sysmsg._
@ -13,13 +12,14 @@ import akka.event.Logging.Error
import akka.util.Unsafe import akka.util.Unsafe
import akka.actor._ import akka.actor._
import akka.serialization.{ DisabledJavaSerializer, SerializationExtension, Serializers } import akka.serialization.{ DisabledJavaSerializer, SerializationExtension, Serializers }
import scala.util.control.{ NoStackTrace, NonFatal } import scala.util.control.{ NoStackTrace, NonFatal }
import scala.util.control.Exception.Catcher import scala.util.control.Exception.Catcher
import akka.dispatch.MailboxType import akka.dispatch.MailboxType
import akka.dispatch.ProducesMessageQueue import akka.dispatch.ProducesMessageQueue
import akka.dispatch.UnboundedMailbox import akka.dispatch.UnboundedMailbox
import akka.serialization.Serialization import akka.serialization.Serialization
import com.github.ghik.silencer.silent
@SerialVersionUID(1L) @SerialVersionUID(1L)
final case class SerializationCheckFailedException private (msg: Object, cause: Throwable) final case class SerializationCheckFailedException private (msg: Object, cause: Throwable)
@ -30,7 +30,7 @@ final case class SerializationCheckFailedException private (msg: Object, cause:
private[akka] trait Dispatch { this: ActorCell => private[akka] trait Dispatch { this: ActorCell =>
@volatile private var _mailboxDoNotCallMeDirectly @silent @volatile private var _mailboxDoNotCallMeDirectly
: Mailbox = _ //This must be volatile since it isn't protected by the mailbox status : Mailbox = _ //This must be volatile since it isn't protected by the mailbox status
@inline final def mailbox: Mailbox = @inline final def mailbox: Mailbox =

View file

@ -12,6 +12,7 @@ import akka.dispatch.sysmsg._
import akka.event.Logging import akka.event.Logging
import akka.event.Logging.Debug import akka.event.Logging.Debug
import akka.event.Logging.Error import akka.event.Logging.Error
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.duration.Duration import scala.concurrent.duration.Duration
import scala.util.control.Exception._ import scala.util.control.Exception._

View file

@ -13,6 +13,7 @@ import akka.dispatch.sysmsg._
import akka.event.EventStream import akka.event.EventStream
import akka.event.Logging.{ Debug, Error, LogEventException } import akka.event.Logging.{ Debug, Error, LogEventException }
import akka.util.{ unused, Index, Unsafe } import akka.util.{ unused, Index, Unsafe }
import com.github.ghik.silencer.silent
import com.typesafe.config.Config import com.typesafe.config.Config
import scala.annotation.tailrec import scala.annotation.tailrec
@ -96,8 +97,8 @@ abstract class MessageDispatcher(val configurator: MessageDispatcherConfigurator
val mailboxes = prerequisites.mailboxes val mailboxes = prerequisites.mailboxes
val eventStream = prerequisites.eventStream val eventStream = prerequisites.eventStream
@volatile private[this] var _inhabitantsDoNotCallMeDirectly: Long = _ // DO NOT TOUCH! @silent @volatile private[this] var _inhabitantsDoNotCallMeDirectly: Long = _ // DO NOT TOUCH!
@volatile private[this] var _shutdownScheduleDoNotCallMeDirectly: Int = _ // DO NOT TOUCH! @silent @volatile private[this] var _shutdownScheduleDoNotCallMeDirectly: Int = _ // DO NOT TOUCH!
private final def addInhabitants(add: Long): Long = { private final def addInhabitants(add: Long): Long = {
val old = Unsafe.instance.getAndAddLong(this, inhabitantsOffset, add) val old = Unsafe.instance.getAndAddLong(this, inhabitantsOffset, add)

View file

@ -9,10 +9,13 @@ import akka.actor.ActorCell
import akka.event.Logging import akka.event.Logging
import akka.dispatch.sysmsg.SystemMessage import akka.dispatch.sysmsg.SystemMessage
import java.util.concurrent.{ ExecutorService, RejectedExecutionException } import java.util.concurrent.{ ExecutorService, RejectedExecutionException }
import scala.concurrent.duration.Duration import scala.concurrent.duration.Duration
import scala.concurrent.duration.FiniteDuration import scala.concurrent.duration.FiniteDuration
import java.util.concurrent.atomic.AtomicReferenceFieldUpdater import java.util.concurrent.atomic.AtomicReferenceFieldUpdater
import com.github.ghik.silencer.silent
/** /**
* The event-based ``Dispatcher`` binds a set of Actors to a thread pool backed up by a * The event-based ``Dispatcher`` binds a set of Actors to a thread pool backed up by a
* `BlockingQueue`. * `BlockingQueue`.
@ -41,6 +44,11 @@ class Dispatcher(
def copy(): LazyExecutorServiceDelegate = new LazyExecutorServiceDelegate(factory) def copy(): LazyExecutorServiceDelegate = new LazyExecutorServiceDelegate(factory)
} }
/**
* At first glance this var does not seem to be updated anywhere, but in
* fact it is, via the esUpdater [[AtomicReferenceFieldUpdater]] below.
*/
@silent
@volatile private var executorServiceDelegate: LazyExecutorServiceDelegate = @volatile private var executorServiceDelegate: LazyExecutorServiceDelegate =
new LazyExecutorServiceDelegate(executorServiceFactoryProvider.createExecutorServiceFactory(id, threadFactory)) new LazyExecutorServiceDelegate(executorServiceFactoryProvider.createExecutorServiceFactory(id, threadFactory))

View file

@ -5,12 +5,15 @@
package akka.dispatch package akka.dispatch
import java.util.concurrent.{ ConcurrentHashMap, ThreadFactory } import java.util.concurrent.{ ConcurrentHashMap, ThreadFactory }
import com.typesafe.config.{ Config, ConfigFactory } import com.typesafe.config.{ Config, ConfigFactory }
import akka.actor.{ ActorSystem, DynamicAccess, Scheduler } import akka.actor.{ ActorSystem, DynamicAccess, Scheduler }
import akka.event.Logging.Warning import akka.event.Logging.Warning
import akka.event.EventStream import akka.event.EventStream
import akka.ConfigurationException import akka.ConfigurationException
import akka.util.Helpers.ConfigOps import akka.util.Helpers.ConfigOps
import com.github.ghik.silencer.silent
import scala.concurrent.ExecutionContext import scala.concurrent.ExecutionContext
/** /**
@ -238,6 +241,7 @@ private[akka] object BalancingDispatcherConfigurator {
* Returns the same dispatcher instance for each invocation * Returns the same dispatcher instance for each invocation
* of the `dispatcher()` method. * of the `dispatcher()` method.
*/ */
@silent
class BalancingDispatcherConfigurator(_config: Config, _prerequisites: DispatcherPrerequisites) class BalancingDispatcherConfigurator(_config: Config, _prerequisites: DispatcherPrerequisites)
extends MessageDispatcherConfigurator(BalancingDispatcherConfigurator.amendConfig(_config), _prerequisites) { extends MessageDispatcherConfigurator(BalancingDispatcherConfigurator.amendConfig(_config), _prerequisites) {

View file

@ -18,6 +18,7 @@ import java.util.concurrent.CompletableFuture
import akka.compat import akka.compat
import akka.util.unused import akka.util.unused
import com.github.ghik.silencer.silent
/** /**
* ExecutionContexts is the Java API for ExecutionContexts * ExecutionContexts is the Java API for ExecutionContexts
@ -240,6 +241,7 @@ object japi {
* *
* Java API * Java API
*/ */
@silent
abstract class OnSuccess[-T] extends japi.CallbackBridge[T] { abstract class OnSuccess[-T] extends japi.CallbackBridge[T] {
protected final override def internal(result: T) = onSuccess(result) protected final override def internal(result: T) = onSuccess(result)
@ -257,6 +259,7 @@ abstract class OnSuccess[-T] extends japi.CallbackBridge[T] {
* *
* Java API * Java API
*/ */
@silent
abstract class OnFailure extends japi.CallbackBridge[Throwable] { abstract class OnFailure extends japi.CallbackBridge[Throwable] {
protected final override def internal(failure: Throwable) = onFailure(failure) protected final override def internal(failure: Throwable) = onFailure(failure)
@ -274,6 +277,7 @@ abstract class OnFailure extends japi.CallbackBridge[Throwable] {
* *
* Java API * Java API
*/ */
@silent
abstract class OnComplete[-T] extends japi.CallbackBridge[Try[T]] { abstract class OnComplete[-T] extends japi.CallbackBridge[Try[T]] {
protected final override def internal(value: Try[T]): Unit = value match { protected final override def internal(value: Try[T]): Unit = value match {
case Failure(t) => onComplete(t, null.asInstanceOf[T]) case Failure(t) => onComplete(t, null.asInstanceOf[T])
@ -296,6 +300,7 @@ abstract class OnComplete[-T] extends japi.CallbackBridge[Try[T]] {
* *
* Java API * Java API
*/ */
@silent
abstract class Recover[+T] extends japi.RecoverBridge[T] { abstract class Recover[+T] extends japi.RecoverBridge[T] {
protected final override def internal(result: Throwable): T = recover(result) protected final override def internal(result: Throwable): T = recover(result)
@ -349,6 +354,7 @@ object Filter {
* SAM (Single Abstract Method) class * SAM (Single Abstract Method) class
* Java API * Java API
*/ */
@silent
abstract class Foreach[-T] extends japi.UnitFunctionBridge[T] { abstract class Foreach[-T] extends japi.UnitFunctionBridge[T] {
override final def internal(t: T): Unit = each(t) override final def internal(t: T): Unit = each(t)

View file

@ -8,6 +8,9 @@ import akka.actor.{ ActorRef, ActorSystem }
import akka.event.Logging.simpleName import akka.event.Logging.simpleName
import akka.util.Subclassification import akka.util.Subclassification
import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicReference
import com.github.ghik.silencer.silent
import scala.annotation.tailrec import scala.annotation.tailrec
/** /**
@ -36,6 +39,8 @@ class EventStream(sys: ActorSystem, private val debug: Boolean) extends LoggingB
protected def classify(event: Any): Class[_] = event.getClass protected def classify(event: Any): Class[_] = event.getClass
// TODO consider avoiding the deprecated `isTerminated`?
@silent
protected def publish(event: Any, subscriber: ActorRef) = { protected def publish(event: Any, subscriber: ActorRef) = {
if (sys == null && subscriber.isTerminated) unsubscribe(subscriber) if (sys == null && subscriber.isTerminated) unsubscribe(subscriber)
else subscriber ! event else subscriber ! event

View file

@ -14,6 +14,7 @@ import akka.dispatch.RequiresMessageQueue
import akka.event.Logging._ import akka.event.Logging._
import akka.util.{ unused, Helpers, ReentrantGuard } import akka.util.{ unused, Helpers, ReentrantGuard }
import akka.{ AkkaException, ConfigurationException } import akka.{ AkkaException, ConfigurationException }
import com.github.ghik.silencer.silent
import scala.annotation.implicitNotFound import scala.annotation.implicitNotFound
import scala.collection.immutable import scala.collection.immutable
@ -166,6 +167,7 @@ trait LoggingBus extends ActorEventBus {
* Internal Akka use only * Internal Akka use only
*/ */
private[akka] def stopDefaultLoggers(system: ActorSystem): Unit = { private[akka] def stopDefaultLoggers(system: ActorSystem): Unit = {
@silent
val level = _logLevel // volatile access before reading loggers val level = _logLevel // volatile access before reading loggers
if (!(loggers contains StandardOutLogger)) { if (!(loggers contains StandardOutLogger)) {
setUpStdoutLogger(system.settings) setUpStdoutLogger(system.settings)
@ -1455,12 +1457,16 @@ trait LoggingAdapter {
* there are more than four arguments. * there are more than four arguments.
*/ */
private def format1(t: String, arg: Any): String = arg match { private def format1(t: String, arg: Any): String = arg match {
case a: Array[_] if !a.getClass.getComponentType.isPrimitive => format(t, a: _*) case a: Array[_] if !a.getClass.getComponentType.isPrimitive => formatImpl(t, a.toSeq)
case a: Array[_] => format(t, a.map(_.asInstanceOf[AnyRef]): _*) case a: Array[_] => formatImpl(t, a.map(_.asInstanceOf[AnyRef]).toSeq)
case x => format(t, x) case x => format(t, x)
} }
def format(t: String, arg: Any*): String = { def format(t: String, arg: Any*): String = {
formatImpl(t, arg)
}
private def formatImpl(t: String, arg: Seq[Any]): String = {
val sb = new java.lang.StringBuilder(64) val sb = new java.lang.StringBuilder(64)
var p = 0 var p = 0
var startIndex = 0 var startIndex = 0
@ -1498,6 +1504,11 @@ trait LoggingFilter {
def isDebugEnabled(logClass: Class[_], logSource: String): Boolean def isDebugEnabled(logClass: Class[_], logSource: String): Boolean
} }
/**
* In retrospect should have been abstract, but we cannot change that
* without breaking binary compatibility
*/
@silent
trait LoggingFilterWithMarker extends LoggingFilter { trait LoggingFilterWithMarker extends LoggingFilter {
def isErrorEnabled(logClass: Class[_], logSource: String, marker: LogMarker): Boolean = def isErrorEnabled(logClass: Class[_], logSource: String, marker: LogMarker): Boolean =
isErrorEnabled(logClass, logSource) isErrorEnabled(logClass, logSource)
@ -1883,8 +1894,8 @@ class MarkerLoggingAdapter(
// Copy of LoggingAdapter.format1 due to binary compatibility restrictions // Copy of LoggingAdapter.format1 due to binary compatibility restrictions
private def format1(t: String, arg: Any): String = arg match { private def format1(t: String, arg: Any): String = arg match {
case a: Array[_] if !a.getClass.getComponentType.isPrimitive => format(t, a: _*) case a: Array[_] if !a.getClass.getComponentType.isPrimitive => format(t, a.toIndexedSeq)
case a: Array[_] => format(t, a.map(_.asInstanceOf[AnyRef]): _*) case a: Array[_] => format(t, a.map(_.asInstanceOf[AnyRef]).toIndexedSeq)
case x => format(t, x) case x => format(t, x)
} }
} }

View file

@ -18,6 +18,7 @@ import akka.util.unused
import scala.collection.immutable import scala.collection.immutable
import akka.util.ccompat._ import akka.util.ccompat._
@ccompatUsedUntil213
abstract class Dns { abstract class Dns {
/** /**

View file

@ -18,9 +18,10 @@ import akka.util.Helpers.Requiring
import akka.util.JavaDurationConverters._ import akka.util.JavaDurationConverters._
import akka.actor._ import akka.actor._
import java.lang.{ Iterable => JIterable } import java.lang.{ Iterable => JIterable }
import java.nio.file.Path import java.nio.file.{ Path, Paths }
import akka.annotation.InternalApi import akka.annotation.InternalApi
import com.github.ghik.silencer.silent
/** /**
* TCP Extension for Akkas IO layer. * TCP Extension for Akkas IO layer.
@ -116,6 +117,7 @@ object Tcp extends ExtensionId[TcpExt] with ExtensionIdProvider {
* @param localAddress optionally specifies a specific address to bind to * @param localAddress optionally specifies a specific address to bind to
* @param options Please refer to the `Tcp.SO` object for a list of all supported options. * @param options Please refer to the `Tcp.SO` object for a list of all supported options.
*/ */
@silent
final case class Connect( final case class Connect(
remoteAddress: InetSocketAddress, remoteAddress: InetSocketAddress,
localAddress: Option[InetSocketAddress] = None, localAddress: Option[InetSocketAddress] = None,
@ -143,6 +145,7 @@ object Tcp extends ExtensionId[TcpExt] with ExtensionIdProvider {
* *
* @param options Please refer to the `Tcp.SO` object for a list of all supported options. * @param options Please refer to the `Tcp.SO` object for a list of all supported options.
*/ */
@silent
final case class Bind( final case class Bind(
handler: ActorRef, handler: ActorRef,
localAddress: InetSocketAddress, localAddress: InetSocketAddress,
@ -867,7 +870,20 @@ object TcpMessage {
* a particular write has been sent by the O/S. * a particular write has been sent by the O/S.
*/ */
def writeFile(filePath: String, position: Long, count: Long, ack: Event): Command = def writeFile(filePath: String, position: Long, count: Long, ack: Event): Command =
WriteFile(filePath, position, count, ack) WritePath(Paths.get(filePath), position, count, ack)
/**
* Write `count` bytes starting at `position` from file at `filePath` to the connection.
* The count must be &gt; 0. The connection actor will reply with a [[Tcp.CommandFailed]]
* message if the write could not be enqueued. If [[Tcp.SimpleWriteCommand#wantsAck]]
* returns true, the connection actor will reply with the supplied [[Tcp.SimpleWriteCommand#ack]]
* token once the write has been successfully enqueued to the O/S kernel.
* <b>Note that this does not in any way guarantee that the data will be
* or have been sent!</b> Unfortunately there is no way to determine whether
* a particular write has been sent by the O/S.
*/
def writePath(filePath: Path, position: Long, count: Long, ack: Event): Command =
WritePath(filePath, position, count, ack)
/** /**
* When `useResumeWriting` is in effect as was indicated in the [[Tcp.Register]] message * When `useResumeWriting` is in effect as was indicated in the [[Tcp.Register]] message

View file

@ -17,6 +17,7 @@ import akka.io.Inet.SocketOption
import akka.io.SelectionHandler._ import akka.io.SelectionHandler._
import akka.io.Tcp._ import akka.io.Tcp._
import akka.util.ByteString import akka.util.ByteString
import com.github.ghik.silencer.silent
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.collection.immutable import scala.collection.immutable
@ -202,6 +203,7 @@ private[io] abstract class TcpConnection(val tcp: TcpExt, val channel: SocketCha
// AUXILIARIES and IMPLEMENTATION // AUXILIARIES and IMPLEMENTATION
/** used in subclasses to start the common machinery above once a channel is connected */ /** used in subclasses to start the common machinery above once a channel is connected */
@silent
def completeConnect( def completeConnect(
registration: ChannelRegistration, registration: ChannelRegistration,
commander: ActorRef, commander: ActorRef,
@ -415,7 +417,8 @@ private[io] abstract class TcpConnection(val tcp: TcpExt, val channel: SocketCha
case Write(data, ack) if data.nonEmpty => PendingBufferWrite(commander, data, ack, tail) case Write(data, ack) if data.nonEmpty => PendingBufferWrite(commander, data, ack, tail)
case WriteFile(path, offset, count, ack) => case WriteFile(path, offset, count, ack) =>
PendingWriteFile(commander, Paths.get(path), offset, count, ack, tail) PendingWriteFile(commander, Paths.get(path), offset, count, ack, tail)
case WritePath(path, offset, count, ack) => PendingWriteFile(commander, path, offset, count, ack, tail) case WritePath(path, offset, count, ack) =>
PendingWriteFile(commander, path, offset, count, ack, tail)
case CompoundWrite(h, t) => create(h, t) case CompoundWrite(h, t) => create(h, t)
case x @ Write(_, ack) => // empty write with either an ACK or a non-standard NoACK case x @ Write(_, ack) => // empty write with either an ACK or a non-standard NoACK
if (x.wantsAck) commander ! ack if (x.wantsAck) commander ! ack

View file

@ -5,9 +5,11 @@
package akka.io package akka.io
import java.nio.channels.SocketChannel import java.nio.channels.SocketChannel
import scala.collection.immutable import scala.collection.immutable
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.io.Inet.SocketOption import akka.io.Inet.SocketOption
import com.github.ghik.silencer.silent
/** /**
* An actor handling the connection state machine for an incoming, already connected * An actor handling the connection state machine for an incoming, already connected
@ -15,6 +17,7 @@ import akka.io.Inet.SocketOption
* *
* INTERNAL API * INTERNAL API
*/ */
@silent
private[io] class TcpIncomingConnection( private[io] class TcpIncomingConnection(
_tcp: TcpExt, _tcp: TcpExt,
_channel: SocketChannel, _channel: SocketChannel,

View file

@ -6,13 +6,16 @@ package akka.io
import java.net.DatagramSocket import java.net.DatagramSocket
import java.net.InetSocketAddress import java.net.InetSocketAddress
import com.typesafe.config.Config import com.typesafe.config.Config
import scala.collection.immutable import scala.collection.immutable
import akka.io.Inet.{ SoJavaFactories, SocketOption } import akka.io.Inet.{ SoJavaFactories, SocketOption }
import akka.util.Helpers.Requiring import akka.util.Helpers.Requiring
import akka.util.ByteString import akka.util.ByteString
import akka.actor._ import akka.actor._
import akka.util.ccompat._ import akka.util.ccompat._
import com.github.ghik.silencer.silent
/** /**
* UDP Extension for Akkas IO layer. * UDP Extension for Akkas IO layer.
@ -26,6 +29,7 @@ import akka.util.ccompat._
* *
* The Java API for generating UDP commands is available at [[UdpMessage]]. * The Java API for generating UDP commands is available at [[UdpMessage]].
*/ */
@ccompatUsedUntil213
object Udp extends ExtensionId[UdpExt] with ExtensionIdProvider { object Udp extends ExtensionId[UdpExt] with ExtensionIdProvider {
override def lookup = Udp override def lookup = Udp
@ -94,6 +98,7 @@ object Udp extends ExtensionId[UdpExt] with ExtensionIdProvider {
* The listener actor for the newly bound port will reply with a [[Bound]] * The listener actor for the newly bound port will reply with a [[Bound]]
* message, or the manager will reply with a [[CommandFailed]] message. * message, or the manager will reply with a [[CommandFailed]] message.
*/ */
@silent
final case class Bind( final case class Bind(
handler: ActorRef, handler: ActorRef,
localAddress: InetSocketAddress, localAddress: InetSocketAddress,
@ -117,6 +122,7 @@ object Udp extends ExtensionId[UdpExt] with ExtensionIdProvider {
* The simple sender will not stop itself, you will have to send it a [[akka.actor.PoisonPill]] * The simple sender will not stop itself, you will have to send it a [[akka.actor.PoisonPill]]
* when you want to close the socket. * when you want to close the socket.
*/ */
@silent
case class SimpleSender(options: immutable.Traversable[SocketOption] = Nil) extends Command case class SimpleSender(options: immutable.Traversable[SocketOption] = Nil) extends Command
object SimpleSender extends SimpleSender(Nil) object SimpleSender extends SimpleSender(Nil)

View file

@ -6,12 +6,14 @@ package akka.io
import java.lang.{ Iterable => JIterable } import java.lang.{ Iterable => JIterable }
import java.net.InetSocketAddress import java.net.InetSocketAddress
import scala.collection.immutable import scala.collection.immutable
import akka.io.Inet.SocketOption import akka.io.Inet.SocketOption
import akka.io.Udp.UdpSettings import akka.io.Udp.UdpSettings
import akka.util.ByteString import akka.util.ByteString
import akka.actor._ import akka.actor._
import akka.util.ccompat._ import akka.util.ccompat._
import com.github.ghik.silencer.silent
/** /**
* UDP Extension for Akkas IO layer. * UDP Extension for Akkas IO layer.
@ -25,6 +27,7 @@ import akka.util.ccompat._
* *
* The Java API for generating UDP commands is available at [[UdpConnectedMessage]]. * The Java API for generating UDP commands is available at [[UdpConnectedMessage]].
*/ */
@ccompatUsedUntil213
object UdpConnected extends ExtensionId[UdpConnectedExt] with ExtensionIdProvider { object UdpConnected extends ExtensionId[UdpConnectedExt] with ExtensionIdProvider {
override def lookup = UdpConnected override def lookup = UdpConnected
@ -88,6 +91,7 @@ object UdpConnected extends ExtensionId[UdpConnectedExt] with ExtensionIdProvide
* which is restricted to sending to and receiving from the given `remoteAddress`. * which is restricted to sending to and receiving from the given `remoteAddress`.
* All received datagrams will be sent to the designated `handler` actor. * All received datagrams will be sent to the designated `handler` actor.
*/ */
@silent
final case class Connect( final case class Connect(
handler: ActorRef, handler: ActorRef,
remoteAddress: InetSocketAddress, remoteAddress: InetSocketAddress,

View file

@ -5,16 +5,19 @@
package akka.io package akka.io
import java.nio.channels.DatagramChannel import java.nio.channels.DatagramChannel
import scala.collection.immutable import scala.collection.immutable
import scala.util.control.NonFatal import scala.util.control.NonFatal
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics } import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
import akka.io.Inet.SocketOption import akka.io.Inet.SocketOption
import akka.io.Udp._ import akka.io.Udp._
import akka.actor._ import akka.actor._
import com.github.ghik.silencer.silent
/** /**
* INTERNAL API * INTERNAL API
*/ */
@silent
private[io] class UdpSender( private[io] class UdpSender(
val udp: UdpExt, val udp: UdpExt,
channelRegistry: ChannelRegistry, channelRegistry: ChannelRegistry,

View file

@ -23,6 +23,7 @@ import akka.util.ccompat._
/** INTERNAL API */ /** INTERNAL API */
@InternalApi @InternalApi
@ccompatUsedUntil213
private[dns] final class DnsSettings(system: ExtendedActorSystem, c: Config) { private[dns] final class DnsSettings(system: ExtendedActorSystem, c: Config) {
import DnsSettings._ import DnsSettings._

View file

@ -11,7 +11,8 @@ import akka.actor.{ Actor, ActorLogging, ActorRef, NoSerializationVerificationNe
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.io.dns.{ RecordClass, RecordType, ResourceRecord } import akka.io.dns.{ RecordClass, RecordType, ResourceRecord }
import akka.io.{ IO, Tcp, Udp } import akka.io.{ IO, Tcp, Udp }
import akka.pattern.BackoffSupervisor import akka.pattern.{ BackoffOpts, BackoffSupervisor }
import com.github.ghik.silencer.silent
import scala.collection.{ immutable => im } import scala.collection.{ immutable => im }
import scala.util.Try import scala.util.Try
@ -69,6 +70,10 @@ import scala.concurrent.duration._
Message(id, MessageFlags(), im.Seq(Question(name, recordType, RecordClass.IN))) Message(id, MessageFlags(), im.Seq(Question(name, recordType, RecordClass.IN)))
} }
/**
* Silent to allow map update syntax
*/
@silent
def ready(socket: ActorRef): Receive = { def ready(socket: ActorRef): Receive = {
case DropRequest(id) => case DropRequest(id) =>
log.debug("Dropping request [{}]", id) log.debug("Dropping request [{}]", id)
@ -143,11 +148,12 @@ import scala.concurrent.duration._
def createTcpClient() = { def createTcpClient() = {
context.actorOf( context.actorOf(
BackoffSupervisor.props( BackoffSupervisor.props(
BackoffOpts.onFailure(
Props(classOf[TcpDnsClient], tcp, ns, self), Props(classOf[TcpDnsClient], tcp, ns, self),
childName = "tcpDnsClient", childName = "tcpDnsClient",
minBackoff = 10.millis, minBackoff = 10.millis,
maxBackoff = 20.seconds, maxBackoff = 20.seconds,
randomFactor = 0.1), randomFactor = 0.1)),
"tcpDnsClientSupervisor") "tcpDnsClientSupervisor")
} }
} }

View file

@ -7,6 +7,7 @@ package akka.io.dns.internal
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.io.dns.ResourceRecord import akka.io.dns.ResourceRecord
import akka.util.{ ByteString, ByteStringBuilder } import akka.util.{ ByteString, ByteStringBuilder }
import com.github.ghik.silencer.silent
import scala.collection.GenTraversableOnce import scala.collection.GenTraversableOnce
import scala.collection.immutable.Seq import scala.collection.immutable.Seq
@ -155,6 +156,7 @@ private[internal] object Message {
} }
import scala.language.implicitConversions import scala.language.implicitConversions
@silent
implicit def flattener[T](tried: Try[T]): GenTraversableOnce[T] = implicit def flattener[T](tried: Try[T]): GenTraversableOnce[T] =
if (flags.isTruncated) tried.toOption if (flags.isTruncated) tried.toOption
else else

View file

@ -27,7 +27,7 @@ private[akka] object DomainName {
} }
def parse(it: ByteIterator, msg: ByteString): String = { def parse(it: ByteIterator, msg: ByteString): String = {
val ret = StringBuilder.newBuilder val ret = new StringBuilder()
while (true) { while (true) {
val length = it.getByte val length = it.getByte
if (length == 0) { if (length == 0) {

View file

@ -7,6 +7,7 @@ package akka.japi
import java.util.Collections.{ emptyList, singletonList } import java.util.Collections.{ emptyList, singletonList }
import akka.util.Collections.EmptyImmutableSeq import akka.util.Collections.EmptyImmutableSeq
import com.github.ghik.silencer.silent
import scala.collection.immutable import scala.collection.immutable
import scala.language.implicitConversions import scala.language.implicitConversions
@ -82,6 +83,7 @@ object Pair {
* *
* This class is kept for compatibility, but for future API's please prefer [[akka.japi.function.Creator]]. * This class is kept for compatibility, but for future API's please prefer [[akka.japi.function.Creator]].
*/ */
@silent
@SerialVersionUID(1L) @SerialVersionUID(1L)
trait Creator[T] extends Serializable { trait Creator[T] extends Serializable {
@ -241,7 +243,8 @@ object Util {
/** /**
* Turns an array into an immutable Scala sequence (by copying it). * Turns an array into an immutable Scala sequence (by copying it).
*/ */
def immutableSeq[T](arr: Array[T]): immutable.Seq[T] = if ((arr ne null) && arr.length > 0) Vector(arr: _*) else Nil def immutableSeq[T](arr: Array[T]): immutable.Seq[T] =
if ((arr ne null) && arr.length > 0) arr.toIndexedSeq else Nil
/** /**
* Turns an [[java.lang.Iterable]] into an immutable Scala sequence (by copying it). * Turns an [[java.lang.Iterable]] into an immutable Scala sequence (by copying it).

View file

@ -4,11 +4,14 @@
package akka.japi.function package akka.japi.function
import com.github.ghik.silencer.silent
/** /**
* A Function interface. Used to create first-class-functions is Java. * A Function interface. Used to create first-class-functions is Java.
* `Serializable` is needed to be able to grab line number for Java 8 lambdas. * `Serializable` is needed to be able to grab line number for Java 8 lambdas.
* Supports throwing `Exception` in the apply, which the `java.util.function.Function` counterpart does not. * Supports throwing `Exception` in the apply, which the `java.util.function.Function` counterpart does not.
*/ */
@silent
@SerialVersionUID(1L) @SerialVersionUID(1L)
trait Function[-T, +R] extends java.io.Serializable { trait Function[-T, +R] extends java.io.Serializable {
@throws(classOf[Exception]) @throws(classOf[Exception])
@ -20,6 +23,7 @@ trait Function[-T, +R] extends java.io.Serializable {
* `Serializable` is needed to be able to grab line number for Java 8 lambdas. * `Serializable` is needed to be able to grab line number for Java 8 lambdas.
* Supports throwing `Exception` in the apply, which the `java.util.function.BiFunction` counterpart does not. * Supports throwing `Exception` in the apply, which the `java.util.function.BiFunction` counterpart does not.
*/ */
@silent
@SerialVersionUID(1L) @SerialVersionUID(1L)
trait Function2[-T1, -T2, +R] extends java.io.Serializable { trait Function2[-T1, -T2, +R] extends java.io.Serializable {
@throws(classOf[Exception]) @throws(classOf[Exception])
@ -31,6 +35,7 @@ trait Function2[-T1, -T2, +R] extends java.io.Serializable {
* `Serializable` is needed to be able to grab line number for Java 8 lambdas. * `Serializable` is needed to be able to grab line number for Java 8 lambdas.
* Supports throwing `Exception` in the apply, which the `java.util.function.Consumer` counterpart does not. * Supports throwing `Exception` in the apply, which the `java.util.function.Consumer` counterpart does not.
*/ */
@silent
@SerialVersionUID(1L) @SerialVersionUID(1L)
trait Procedure[-T] extends java.io.Serializable { trait Procedure[-T] extends java.io.Serializable {
@throws(classOf[Exception]) @throws(classOf[Exception])
@ -42,6 +47,7 @@ trait Procedure[-T] extends java.io.Serializable {
* `Serializable` is needed to be able to grab line number for Java 8 lambdas. * `Serializable` is needed to be able to grab line number for Java 8 lambdas.
* Supports throwing `Exception` in the apply, which the `java.util.function.Effect` counterpart does not. * Supports throwing `Exception` in the apply, which the `java.util.function.Effect` counterpart does not.
*/ */
@silent
@SerialVersionUID(1L) @SerialVersionUID(1L)
trait Effect extends java.io.Serializable { trait Effect extends java.io.Serializable {
@ -54,6 +60,7 @@ trait Effect extends java.io.Serializable {
* `Serializable` is needed to be able to grab line number for Java 8 lambdas. * `Serializable` is needed to be able to grab line number for Java 8 lambdas.
* Supports throwing `Exception` in the apply, which the `java.util.function.Predicate` counterpart does not. * Supports throwing `Exception` in the apply, which the `java.util.function.Predicate` counterpart does not.
*/ */
@silent
@SerialVersionUID(1L) @SerialVersionUID(1L)
trait Predicate[-T] extends java.io.Serializable { trait Predicate[-T] extends java.io.Serializable {
def test(param: T): Boolean def test(param: T): Boolean
@ -63,6 +70,7 @@ trait Predicate[-T] extends java.io.Serializable {
* A constructor/factory, takes no parameters but creates a new value of type T every call. * A constructor/factory, takes no parameters but creates a new value of type T every call.
* Supports throwing `Exception` in the apply, which the `java.util.function.Creator` counterpart does not. * Supports throwing `Exception` in the apply, which the `java.util.function.Creator` counterpart does not.
*/ */
@silent
@SerialVersionUID(1L) @SerialVersionUID(1L)
trait Creator[+T] extends Serializable { trait Creator[+T] extends Serializable {

View file

@ -11,6 +11,8 @@ import akka.annotation.InternalApi
import akka.dispatch.sysmsg._ import akka.dispatch.sysmsg._
import akka.util.{ Timeout, Unsafe } import akka.util.{ Timeout, Unsafe }
import com.github.ghik.silencer.silent
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.concurrent.{ ExecutionContext, Future, Promise } import scala.concurrent.{ ExecutionContext, Future, Promise }
import scala.language.implicitConversions import scala.language.implicitConversions
@ -524,9 +526,11 @@ private[akka] final class PromiseActorRef private (
* Stopped => stopped, path not yet created * Stopped => stopped, path not yet created
*/ */
@volatile @volatile
@silent
private[this] var _stateDoNotCallMeDirectly: AnyRef = _ private[this] var _stateDoNotCallMeDirectly: AnyRef = _
@volatile @volatile
@silent
private[this] var _watchedByDoNotCallMeDirectly: Set[ActorRef] = ActorCell.emptyActorRefSet private[this] var _watchedByDoNotCallMeDirectly: Set[ActorRef] = ActorCell.emptyActorRefSet
@inline @inline

View file

@ -8,6 +8,7 @@ import akka.actor.{ OneForOneStrategy, Props, SupervisorStrategy }
import akka.annotation.DoNotInherit import akka.annotation.DoNotInherit
import akka.pattern.internal.{ BackoffOnRestartSupervisor, BackoffOnStopSupervisor } import akka.pattern.internal.{ BackoffOnRestartSupervisor, BackoffOnStopSupervisor }
import akka.util.JavaDurationConverters._ import akka.util.JavaDurationConverters._
import com.github.ghik.silencer.silent
import scala.concurrent.duration.{ Duration, FiniteDuration } import scala.concurrent.duration.{ Duration, FiniteDuration }
@ -565,6 +566,7 @@ trait BackoffOptions {
private[akka] def props: Props private[akka] def props: Props
} }
@silent
private final case class BackoffOptionsImpl( private final case class BackoffOptionsImpl(
backoffType: BackoffType = RestartImpliesFailure, backoffType: BackoffType = RestartImpliesFailure,
childProps: Props, childProps: Props,

View file

@ -24,6 +24,8 @@ import scala.util.control.NonFatal
import scala.util.{ Failure, Success, Try } import scala.util.{ Failure, Success, Try }
import akka.dispatch.ExecutionContexts.sameThreadExecutionContext import akka.dispatch.ExecutionContexts.sameThreadExecutionContext
import com.github.ghik.silencer.silent
import scala.compat.java8.FutureConverters import scala.compat.java8.FutureConverters
/** /**
@ -165,7 +167,8 @@ class CircuitBreaker(
// add the old constructor to make it binary compatible // add the old constructor to make it binary compatible
def this(scheduler: Scheduler, maxFailures: Int, callTimeout: FiniteDuration, resetTimeout: FiniteDuration)( def this(scheduler: Scheduler, maxFailures: Int, callTimeout: FiniteDuration, resetTimeout: FiniteDuration)(
implicit executor: ExecutionContext) = { implicit
executor: ExecutionContext) = {
this(scheduler, maxFailures, callTimeout, resetTimeout, 36500.days, 1.0)(executor) this(scheduler, maxFailures, callTimeout, resetTimeout, 36500.days, 1.0)(executor)
} }
@ -193,12 +196,14 @@ class CircuitBreaker(
* Holds reference to current state of CircuitBreaker - *access only via helper methods* * Holds reference to current state of CircuitBreaker - *access only via helper methods*
*/ */
@volatile @volatile
@silent
private[this] var _currentStateDoNotCallMeDirectly: State = Closed private[this] var _currentStateDoNotCallMeDirectly: State = Closed
/** /**
* Holds reference to current resetTimeout of CircuitBreaker - *access only via helper methods* * Holds reference to current resetTimeout of CircuitBreaker - *access only via helper methods*
*/ */
@volatile @volatile
@silent
private[this] var _currentResetTimeoutDoNotCallMeDirectly: FiniteDuration = resetTimeout private[this] var _currentResetTimeoutDoNotCallMeDirectly: FiniteDuration = resetTimeout
/** /**

View file

@ -6,12 +6,11 @@ package akka.pattern
import akka.actor.{ Actor, ActorRef, Props } import akka.actor.{ Actor, ActorRef, Props }
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.pattern.internal.{ BackoffOnRestartSupervisor, BackoffOnStopSupervisor }
/** /**
* INTERNAL API * INTERNAL API
* *
* Implements basic backoff handling for [[BackoffOnRestartSupervisor]] and [[BackoffOnStopSupervisor]]. * Implements basic backoff handling for [[internal.BackoffOnRestartSupervisor]] and [[internal.BackoffOnStopSupervisor]].
*/ */
@InternalApi private[akka] trait HandleBackoff { @InternalApi private[akka] trait HandleBackoff {
this: Actor => this: Actor =>

View file

@ -13,6 +13,7 @@ import akka.actor.Props
import akka.actor.SupervisorStrategy import akka.actor.SupervisorStrategy
import akka.dispatch.BalancingDispatcherConfigurator import akka.dispatch.BalancingDispatcherConfigurator
import akka.dispatch.Dispatchers import akka.dispatch.Dispatchers
import com.github.ghik.silencer.silent
/** /**
* INTERNAL API * INTERNAL API
@ -25,6 +26,7 @@ private[akka] object BalancingRoutingLogic {
* INTERNAL API * INTERNAL API
* Selects the first routee, balancing will be done by the dispatcher. * Selects the first routee, balancing will be done by the dispatcher.
*/ */
@silent
@SerialVersionUID(1L) @SerialVersionUID(1L)
private[akka] final class BalancingRoutingLogic extends RoutingLogic { private[akka] final class BalancingRoutingLogic extends RoutingLogic {
override def select(message: Any, routees: immutable.IndexedSeq[Routee]): Routee = override def select(message: Any, routees: immutable.IndexedSeq[Routee]): Routee =

View file

@ -10,6 +10,7 @@ import com.typesafe.config.Config
import akka.actor.SupervisorStrategy import akka.actor.SupervisorStrategy
import akka.japi.Util.immutableSeq import akka.japi.Util.immutableSeq
import akka.actor.ActorSystem import akka.actor.ActorSystem
import com.github.ghik.silencer.silent
object BroadcastRoutingLogic { object BroadcastRoutingLogic {
def apply(): BroadcastRoutingLogic = new BroadcastRoutingLogic def apply(): BroadcastRoutingLogic = new BroadcastRoutingLogic
@ -18,6 +19,7 @@ object BroadcastRoutingLogic {
/** /**
* Broadcasts a message to all its routees. * Broadcasts a message to all its routees.
*/ */
@silent
@SerialVersionUID(1L) @SerialVersionUID(1L)
final class BroadcastRoutingLogic extends RoutingLogic { final class BroadcastRoutingLogic extends RoutingLogic {
override def select(message: Any, routees: immutable.IndexedSeq[Routee]): Routee = override def select(message: Any, routees: immutable.IndexedSeq[Routee]): Routee =

View file

@ -20,7 +20,9 @@
package akka.routing package akka.routing
import java.lang.Integer.{ rotateLeft => rotl } import java.lang.Integer.{ rotateLeft => rotl }
import akka.util.ccompat._ import akka.util.ccompat._
import com.github.ghik.silencer.silent
/** /**
* An object designed to generate well-distributed non-cryptographic * An object designed to generate well-distributed non-cryptographic
@ -31,6 +33,7 @@ import akka.util.ccompat._
* incorporate a new integer) to update the values. Only one method * incorporate a new integer) to update the values. Only one method
* needs to be called to finalize the hash. * needs to be called to finalize the hash.
*/ */
@ccompatUsedUntil213
object MurmurHash { object MurmurHash {
// Magic values used for MurmurHash's 32 bit hash. // Magic values used for MurmurHash's 32 bit hash.
// Don't change these without consulting a hashing expert! // Don't change these without consulting a hashing expert!
@ -129,6 +132,7 @@ object MurmurHash {
* where the order of appearance of elements does not matter. * where the order of appearance of elements does not matter.
* This is useful for hashing sets, for example. * This is useful for hashing sets, for example.
*/ */
@silent
def symmetricHash[T](xs: IterableOnce[T], seed: Int): Int = { def symmetricHash[T](xs: IterableOnce[T], seed: Int): Int = {
var a, b, n = 0 var a, b, n = 0
var c = 1 var c = 1

View file

@ -6,11 +6,13 @@ package akka.routing
import scala.collection.immutable import scala.collection.immutable
import java.util.concurrent.ThreadLocalRandom import java.util.concurrent.ThreadLocalRandom
import akka.dispatch.Dispatchers import akka.dispatch.Dispatchers
import com.typesafe.config.Config import com.typesafe.config.Config
import akka.actor.SupervisorStrategy import akka.actor.SupervisorStrategy
import akka.japi.Util.immutableSeq import akka.japi.Util.immutableSeq
import akka.actor.ActorSystem import akka.actor.ActorSystem
import com.github.ghik.silencer.silent
object RandomRoutingLogic { object RandomRoutingLogic {
def apply(): RandomRoutingLogic = new RandomRoutingLogic def apply(): RandomRoutingLogic = new RandomRoutingLogic
@ -19,6 +21,7 @@ object RandomRoutingLogic {
/** /**
* Randomly selects one of the target routees to send a message to * Randomly selects one of the target routees to send a message to
*/ */
@silent
@SerialVersionUID(1L) @SerialVersionUID(1L)
final class RandomRoutingLogic extends RoutingLogic { final class RandomRoutingLogic extends RoutingLogic {
override def select(message: Any, routees: immutable.IndexedSeq[Routee]): Routee = override def select(message: Any, routees: immutable.IndexedSeq[Routee]): Routee =

View file

@ -5,12 +5,14 @@
package akka.routing package akka.routing
import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.atomic.AtomicLong
import scala.collection.immutable import scala.collection.immutable
import akka.dispatch.Dispatchers import akka.dispatch.Dispatchers
import com.typesafe.config.Config import com.typesafe.config.Config
import akka.actor.SupervisorStrategy import akka.actor.SupervisorStrategy
import akka.japi.Util.immutableSeq import akka.japi.Util.immutableSeq
import akka.actor.ActorSystem import akka.actor.ActorSystem
import com.github.ghik.silencer.silent
object RoundRobinRoutingLogic { object RoundRobinRoutingLogic {
def apply(): RoundRobinRoutingLogic = new RoundRobinRoutingLogic def apply(): RoundRobinRoutingLogic = new RoundRobinRoutingLogic
@ -20,6 +22,7 @@ object RoundRobinRoutingLogic {
* Uses round-robin to select a routee. For concurrent calls, * Uses round-robin to select a routee. For concurrent calls,
* round robin is just a best effort. * round robin is just a best effort.
*/ */
@silent
@SerialVersionUID(1L) @SerialVersionUID(1L)
final class RoundRobinRoutingLogic extends RoutingLogic { final class RoundRobinRoutingLogic extends RoutingLogic {
val next = new AtomicLong val next = new AtomicLong

View file

@ -36,6 +36,7 @@ private[akka] object RoutedActorCell {
/** /**
* INTERNAL API * INTERNAL API
*/ */
@ccompatUsedUntil213
private[akka] class RoutedActorCell( private[akka] class RoutedActorCell(
_system: ActorSystemImpl, _system: ActorSystemImpl,
_ref: InternalActorRef, _ref: InternalActorRef,

View file

@ -15,6 +15,7 @@ import akka.actor.UnstartedCell
import akka.dispatch.BalancingDispatcher import akka.dispatch.BalancingDispatcher
import akka.dispatch.MailboxType import akka.dispatch.MailboxType
import akka.dispatch.MessageDispatcher import akka.dispatch.MessageDispatcher
import com.github.ghik.silencer.silent
/** /**
* INTERNAL API * INTERNAL API
@ -22,6 +23,7 @@ import akka.dispatch.MessageDispatcher
* A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to * A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to
* send a message to one (or more) of these actors. * send a message to one (or more) of these actors.
*/ */
@silent
private[akka] class RoutedActorRef( private[akka] class RoutedActorRef(
_system: ActorSystemImpl, _system: ActorSystemImpl,
_routerProps: Props, _routerProps: Props,

View file

@ -17,6 +17,7 @@ import akka.dispatch.Dispatchers
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.japi.Util.immutableSeq import akka.japi.Util.immutableSeq
import akka.util.unused import akka.util.unused
import com.github.ghik.silencer.silent
/** /**
* This trait represents a router factory: it produces the actual router actor * This trait represents a router factory: it produces the actual router actor
@ -36,6 +37,7 @@ import akka.util.unused
* someone tries sending a message to that reference before the constructor of * someone tries sending a message to that reference before the constructor of
* RoutedActorRef has returned, there will be a `NullPointerException`! * RoutedActorRef has returned, there will be a `NullPointerException`!
*/ */
@silent
@SerialVersionUID(1L) @SerialVersionUID(1L)
trait RouterConfig extends Serializable { trait RouterConfig extends Serializable {
@ -372,14 +374,18 @@ case object NoRouter extends NoRouter {
/** /**
* INTERNAL API * INTERNAL API
*/ */
@SerialVersionUID(1L) private[akka] trait RouterManagementMesssage @silent
@SerialVersionUID(1L)
private[akka] trait RouterManagementMesssage
/** /**
* Sending this message to a router will make it send back its currently used routees. * Sending this message to a router will make it send back its currently used routees.
* A [[Routees]] message is sent asynchronously to the "requester" containing information * A [[Routees]] message is sent asynchronously to the "requester" containing information
* about what routees the router is routing over. * about what routees the router is routing over.
*/ */
@SerialVersionUID(1L) abstract class GetRoutees extends RouterManagementMesssage @silent
@SerialVersionUID(1L)
abstract class GetRoutees extends RouterManagementMesssage
@SerialVersionUID(1L) case object GetRoutees extends GetRoutees { @SerialVersionUID(1L) case object GetRoutees extends GetRoutees {

View file

@ -54,9 +54,9 @@ private[akka] final case class ScatterGatherFirstCompletedRoutees(
val promise = Promise[Any]() val promise = Promise[Any]()
routees.foreach { routees.foreach {
case ActorRefRoutee(ref) => case ActorRefRoutee(ref) =>
promise.tryCompleteWith(ref.ask(message)) promise.completeWith(ref.ask(message))
case ActorSelectionRoutee(sel) => case ActorSelectionRoutee(sel) =>
promise.tryCompleteWith(sel.ask(message)) promise.completeWith(sel.ask(message))
case _ => case _ =>
} }

View file

@ -7,12 +7,14 @@ package akka.routing
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.collection.immutable import scala.collection.immutable
import java.util.concurrent.ThreadLocalRandom import java.util.concurrent.ThreadLocalRandom
import com.typesafe.config.Config import com.typesafe.config.Config
import akka.actor.ActorCell import akka.actor.ActorCell
import akka.actor.ActorRefWithCell import akka.actor.ActorRefWithCell
import akka.actor.SupervisorStrategy import akka.actor.SupervisorStrategy
import akka.dispatch.Dispatchers import akka.dispatch.Dispatchers
import akka.actor.ActorSystem import akka.actor.ActorSystem
import com.github.ghik.silencer.silent
object SmallestMailboxRoutingLogic { object SmallestMailboxRoutingLogic {
def apply(): SmallestMailboxRoutingLogic = new SmallestMailboxRoutingLogic def apply(): SmallestMailboxRoutingLogic = new SmallestMailboxRoutingLogic
@ -29,6 +31,7 @@ object SmallestMailboxRoutingLogic {
* since their mailbox size is unknown</li> * since their mailbox size is unknown</li>
* </ul> * </ul>
*/ */
@silent
@SerialVersionUID(1L) @SerialVersionUID(1L)
class SmallestMailboxRoutingLogic extends RoutingLogic { class SmallestMailboxRoutingLogic extends RoutingLogic {
override def select(message: Any, routees: immutable.IndexedSeq[Routee]): Routee = override def select(message: Any, routees: immutable.IndexedSeq[Routee]): Routee =
@ -77,6 +80,8 @@ class SmallestMailboxRoutingLogic extends RoutingLogic {
} }
} }
// TODO should we rewrite this not to use isTerminated?
@silent
protected def isTerminated(a: Routee): Boolean = a match { protected def isTerminated(a: Routee): Boolean = a match {
case ActorRefRoutee(ref) => ref.isTerminated case ActorRefRoutee(ref) => ref.isTerminated
case _ => false case _ => false
@ -174,6 +179,7 @@ class SmallestMailboxRoutingLogic extends RoutingLogic {
* @param routerDispatcher dispatcher to use for the router head actor, which handles * @param routerDispatcher dispatcher to use for the router head actor, which handles
* supervision, death watch and router management messages * supervision, death watch and router management messages
*/ */
@silent
@SerialVersionUID(1L) @SerialVersionUID(1L)
final case class SmallestMailboxPool( final case class SmallestMailboxPool(
val nrOfInstances: Int, val nrOfInstances: Int,

View file

@ -82,9 +82,9 @@ private[akka] final case class TailChoppingRoutees(
if (idx < size) { if (idx < size) {
shuffled(idx) match { shuffled(idx) match {
case ActorRefRoutee(ref) => case ActorRefRoutee(ref) =>
promise.tryCompleteWith(ref.ask(message)) promise.completeWith(ref.ask(message))
case ActorSelectionRoutee(sel) => case ActorSelectionRoutee(sel) =>
promise.tryCompleteWith(sel.ask(message)) promise.completeWith(sel.ask(message))
case _ => case _ =>
} }
} }

View file

@ -24,6 +24,7 @@ import java.util.NoSuchElementException
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.util.ccompat._ import akka.util.ccompat._
@ccompatUsedUntil213
object Serialization { object Serialization {
/** /**

View file

@ -16,6 +16,7 @@ import akka.actor.Extension
import akka.actor.ExtensionId import akka.actor.ExtensionId
import akka.actor.ExtensionIdProvider import akka.actor.ExtensionIdProvider
import akka.event.Logging import akka.event.Logging
import com.github.ghik.silencer.silent
/** /**
* Akka extension that extracts [[ManifestInfo.Version]] information from META-INF/MANIFEST.MF in jar files * Akka extension that extracts [[ManifestInfo.Version]] information from META-INF/MANIFEST.MF in jar files
@ -162,6 +163,7 @@ final class ManifestInfo(val system: ExtendedActorSystem) extends Extension {
* Verify that the version is the same for all given artifacts. * Verify that the version is the same for all given artifacts.
*/ */
def checkSameVersion(productName: String, dependencies: immutable.Seq[String], logWarning: Boolean): Boolean = { def checkSameVersion(productName: String, dependencies: immutable.Seq[String], logWarning: Boolean): Boolean = {
@silent
val filteredVersions = versions.filterKeys(dependencies.toSet) val filteredVersions = versions.filterKeys(dependencies.toSet)
val values = filteredVersions.values.toSet val values = filteredVersions.values.toSet
if (values.size > 1) { if (values.size > 1) {

View file

@ -78,6 +78,7 @@ private[akka] object SubclassifiedIndex {
* cache, e.g. HashMap, is faster than tree traversal which must use linear * cache, e.g. HashMap, is faster than tree traversal which must use linear
* scan at each level. Therefore, no value traversals are published. * scan at each level. Therefore, no value traversals are published.
*/ */
@ccompatUsedUntil213
private[akka] class SubclassifiedIndex[K, V] private (protected var values: Set[V])(implicit sc: Subclassification[K]) { private[akka] class SubclassifiedIndex[K, V] private (protected var values: Set[V])(implicit sc: Subclassification[K]) {
import SubclassifiedIndex._ import SubclassifiedIndex._

View file

@ -6,9 +6,11 @@ package akka.cluster.metrics
import java.util.Arrays import java.util.Arrays
import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicReference
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.collection.immutable import scala.collection.immutable
import java.util.concurrent.ThreadLocalRandom import java.util.concurrent.ThreadLocalRandom
import com.typesafe.config.Config import com.typesafe.config.Config
import akka.actor.Actor import akka.actor.Actor
import akka.actor.ActorSystem import akka.actor.ActorSystem
@ -22,6 +24,7 @@ import akka.dispatch.Dispatchers
import akka.japi.Util.immutableSeq import akka.japi.Util.immutableSeq
import akka.routing._ import akka.routing._
import akka.cluster.routing.ClusterRouterSettingsBase import akka.cluster.routing.ClusterRouterSettingsBase
import com.github.ghik.silencer.silent
/** /**
* Load balancing of messages to cluster nodes based on cluster metric data. * Load balancing of messages to cluster nodes based on cluster metric data.
@ -416,6 +419,7 @@ object MetricsSelector {
/** /**
* A MetricsSelector is responsible for producing weights from the node metrics. * A MetricsSelector is responsible for producing weights from the node metrics.
*/ */
@silent
@SerialVersionUID(1L) @SerialVersionUID(1L)
trait MetricsSelector extends Serializable { trait MetricsSelector extends Serializable {
@ -429,6 +433,7 @@ trait MetricsSelector extends Serializable {
* A MetricsSelector producing weights from remaining capacity. * A MetricsSelector producing weights from remaining capacity.
* The weights are typically proportional to the remaining capacity. * The weights are typically proportional to the remaining capacity.
*/ */
@silent
@SerialVersionUID(1L) @SerialVersionUID(1L)
abstract class CapacityMetricsSelector extends MetricsSelector { abstract class CapacityMetricsSelector extends MetricsSelector {

View file

@ -94,7 +94,7 @@ abstract class AbstractTransportAdapter(protected val wrappedTransport: Transpor
(listenAddress, listenerPromise) <- wrappedTransport.listen (listenAddress, listenerPromise) <- wrappedTransport.listen
// Enforce ordering between the signalling of "listen ready" to upstream // Enforce ordering between the signalling of "listen ready" to upstream
// and initialization happening in interceptListen // and initialization happening in interceptListen
_ <- listenerPromise.tryCompleteWith(interceptListen(listenAddress, upstreamListenerPromise.future)).future _ <- listenerPromise.completeWith(interceptListen(listenAddress, upstreamListenerPromise.future)).future
} yield (augmentScheme(listenAddress), upstreamListenerPromise) } yield (augmentScheme(listenAddress), upstreamListenerPromise)
} }

View file

@ -8,10 +8,6 @@ import sbt._
import Keys.{scalacOptions, _} import Keys.{scalacOptions, _}
import sbt.plugins.JvmPlugin import sbt.plugins.JvmPlugin
/**
* Initial tests found:
* `akka-actor` 151 errors with `-Xfatal-warnings`, 6 without the flag
*/
object AkkaDisciplinePlugin extends AutoPlugin with ScalafixSupport { object AkkaDisciplinePlugin extends AutoPlugin with ScalafixSupport {
import scoverage.ScoverageKeys._ import scoverage.ScoverageKeys._
@ -22,6 +18,7 @@ object AkkaDisciplinePlugin extends AutoPlugin with ScalafixSupport {
override lazy val projectSettings = disciplineSettings override lazy val projectSettings = disciplineSettings
val fatalWarningsFor = Set( val fatalWarningsFor = Set(
"akka-actor",
"akka-discovery", "akka-discovery",
"akka-distributed-data", "akka-distributed-data",
"akka-coordination", "akka-coordination",
@ -58,24 +55,14 @@ object AkkaDisciplinePlugin extends AutoPlugin with ScalafixSupport {
scalaFixSettings ++ scalaFixSettings ++
silencerSettings ++ silencerSettings ++
scoverageSettings ++ Seq( scoverageSettings ++ Seq(
Compile / scalacOptions ++= (if (strictProjects.contains(name.value)) {
disciplineScalacOptions
} else {
disciplineScalacOptions -- undisciplineScalacOptions
}).toSeq,
Compile / scalacOptions ++= ( Compile / scalacOptions ++= (
if (fatalWarningsFor(name.value)) Seq("-Xfatal-warnings") if (fatalWarningsFor(name.value)) Seq("-Xfatal-warnings")
else Seq.empty else Seq.empty
), ),
Compile / console / scalacOptions --= Seq("-deprecation", "-Xfatal-warnings", "-Xlint", "-Ywarn-unused:imports"), Compile / console / scalacOptions --= Seq("-deprecation", "-Xfatal-warnings", "-Xlint", "-Ywarn-unused:imports"),
// Discipline is not needed for the docs compilation run (which uses Compile / scalacOptions ++= (CrossVersion.partialVersion(scalaVersion.value) match {
// different compiler phases from the regular run), and in particular
// '-Ywarn-unused:explicits' breaks 'sbt ++2.13.0-M5 akka-actor/doc'
// https://github.com/akka/akka/issues/26119
Compile / doc / scalacOptions --= disciplineScalacOptions.toSeq,
Compile / scalacOptions --= (CrossVersion.partialVersion(scalaVersion.value) match {
case Some((2, 13)) => case Some((2, 13)) =>
Seq( disciplineScalacOptions -- Set(
"-Ywarn-inaccessible", "-Ywarn-inaccessible",
"-Ywarn-infer-any", "-Ywarn-infer-any",
"-Ywarn-nullary-override", "-Ywarn-nullary-override",
@ -84,18 +71,30 @@ object AkkaDisciplinePlugin extends AutoPlugin with ScalafixSupport {
"-Yno-adapted-args", "-Yno-adapted-args",
) )
case Some((2, 12)) => case Some((2, 12)) =>
Nil disciplineScalacOptions
case Some((2, 11)) => case Some((2, 11)) =>
Seq("-Ywarn-extra-implicit", "-Ywarn-unused:_") disciplineScalacOptions ++ Set("-language:existentials") -- Set(
"-Ywarn-extra-implicit",
"-Ywarn-unused:_",
"-Ypartial-unification",
)
case _ => case _ =>
Nil Nil
}), }).toSeq,
Compile / doc / scalacOptions ++= (CrossVersion.partialVersion(scalaVersion.value) match { Compile / doc / scalacOptions ++= (CrossVersion.partialVersion(scalaVersion.value) match {
case Some((2, 11)) => case Some((2, 11)) =>
Seq("-no-link-warnings") Seq("-no-link-warnings")
case _ => case _ =>
Seq.empty Seq.empty
}), }),
Compile / scalacOptions --=
(if (strictProjects.contains(name.value)) Seq.empty
else undisciplineScalacOptions.toSeq),
// Discipline is not needed for the docs compilation run (which uses
// different compiler phases from the regular run), and in particular
// '-Ywarn-unused:explicits' breaks 'sbt ++2.13.0-M5 akka-actor/doc'
// https://github.com/akka/akka/issues/26119
Compile / doc / scalacOptions --= disciplineScalacOptions.toSeq :+ "-Xfatal-warnings",
) )
/** /**