diff --git a/akka-actor/src/main/scala/akka/AkkaException.scala b/akka-actor/src/main/scala/akka/AkkaException.scala index 79d78b9d39..e5b0cb6c80 100644 --- a/akka-actor/src/main/scala/akka/AkkaException.scala +++ b/akka-actor/src/main/scala/akka/AkkaException.scala @@ -5,19 +5,26 @@ package akka object AkkaException { - + //FIXME DOC def toStringWithStackTrace(throwable: Throwable): String = throwable match { case null ⇒ "Unknown Throwable: was 'null'" case ae: AkkaException ⇒ ae.toLongString case e ⇒ "%s:%s\n%s" format (e.getClass.getName, e.getMessage, stackTraceToString(e)) } - def stackTraceToString(throwable: Throwable): String = { - val trace = throwable.getStackTrace - val sb = new StringBuilder - for (i ← 0 until trace.length) - sb.append("\tat %s\n" format trace(i)) - sb.toString + /** + * Returns the given Throwables stack trace as a String, or the empty String if no trace is found + * @param throwable + * @return + */ + def stackTraceToString(throwable: Throwable): String = throwable.getStackTrace match { + case null ⇒ "" + case x if x.length == 0 ⇒ "" + case trace ⇒ + val sb = new StringBuilder + for (i ← 0 until trace.length) + sb.append("\tat %s\n" format trace(i)) + sb.toString } } @@ -32,17 +39,15 @@ object AkkaException { */ //TODO add @SerialVersionUID(1L) when SI-4804 is fixed class AkkaException(message: String = "", cause: Throwable = null) extends RuntimeException(message, cause) with Serializable { - lazy val uuid = java.util.UUID.randomUUID().toString - - override lazy val toString = - "%s:%s\n[%s]".format(getClass.getName, message, uuid) - - lazy val toLongString = - "%s:%s\n[%s]\n%s".format(getClass.getName, message, uuid, stackTraceToString) - def this(msg: String) = this(msg, null) - def stackTraceToString = AkkaException.stackTraceToString(this) + lazy val uuid = java.util.UUID.randomUUID().toString + + override def toString: String = "%s:%s\n[%s]".format(getClass.getName, message, uuid) + + def toLongString: String = "%s:%s\n[%s]\n%s".format(getClass.getName, message, uuid, stackTraceToString) + + def stackTraceToString: String = AkkaException.stackTraceToString(this) } /** diff --git a/akka-actor/src/main/scala/akka/experimental.scala b/akka-actor/src/main/scala/akka/experimental.scala deleted file mode 100644 index aef3cb5c85..0000000000 --- a/akka-actor/src/main/scala/akka/experimental.scala +++ /dev/null @@ -1,19 +0,0 @@ -/** - * Copyright (C) 2009-2012 Typesafe Inc. - */ - -package akka - -import annotation.target._ - -/** - * This annotation marks a feature which is not yet considered stable and may - * change or be removed in a future release. - * - * @since 1.2 - */ -@getter -@setter -@beanGetter -@beanSetter -final class experimental(since: String) extends annotation.StaticAnnotation diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index c3db8293d2..f27919d316 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -47,12 +47,11 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup ref: InternalActorRef, props: Props, supervisor: InternalActorRef, - receiveTimeout: Option[Duration]): ActorCell = - { - val cell = super.newActorCell(system, ref, props, supervisor, receiveTimeout) - Unsafe.instance.monitorEnter(cell) - cell - } + receiveTimeout: Option[Duration]): ActorCell = { + val cell = super.newActorCell(system, ref, props, supervisor, receiveTimeout) + Unsafe.instance.monitorEnter(cell) + cell + } private[akka] val routerConfig = _props.routerConfig private[akka] val routeeProps = _props.copy(routerConfig = NoRouter) @@ -303,8 +302,8 @@ trait Router extends Actor { final def receive = ({ case Router.Resize ⇒ - try ref.routerConfig.resizer foreach (_.resize(ref.routeeProps, ref.routeeProvider)) - finally assert(ref.resizeInProgress.getAndSet(false)) + val ab = ref.resizeInProgress + if (ab.get) try ref.routerConfig.resizer foreach (_.resize(ref.routeeProps, ref.routeeProvider)) finally ab.set(false) case Terminated(child) ⇒ ref.removeRoutees(IndexedSeq(child)) @@ -319,6 +318,9 @@ trait Router extends Actor { } } +/** + * INTERNAL API + */ private object Router { case object Resize @@ -372,9 +374,9 @@ case class Destination(sender: ActorRef, recipient: ActorRef) //TODO add @SerialVersionUID(1L) when SI-4804 is fixed abstract class NoRouter extends RouterConfig case object NoRouter extends NoRouter { - def createRoute(props: Props, routeeProvider: RouteeProvider): Route = null + def createRoute(props: Props, routeeProvider: RouteeProvider): Route = null // FIXME, null, really?? def routerDispatcher: String = "" - def supervisorStrategy = null + def supervisorStrategy = null // FIXME null, really?? override def withFallback(other: RouterConfig): RouterConfig = other /** @@ -404,9 +406,7 @@ case object FromConfig extends FromConfig { //TODO add @SerialVersionUID(1L) when SI-4804 is fixed class FromConfig(val routerDispatcher: String = Dispatchers.DefaultDispatcherId) extends RouterConfig - with Product - with Serializable - with Equals { + with Serializable { def this() = this(Dispatchers.DefaultDispatcherId) @@ -414,38 +414,6 @@ class FromConfig(val routerDispatcher: String = Dispatchers.DefaultDispatcherId) throw new ConfigurationException("router " + routeeProvider.context.self + " needs external configuration from file (e.g. application.conf)") def supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy - - // open-coded case class to preserve binary compatibility, all deprecated for 2.1 - @deprecated("FromConfig does not make sense as case class", "2.0.1") - override def productPrefix = "FromConfig" - - @deprecated("FromConfig does not make sense as case class", "2.0.1") - def productArity = 1 - - @deprecated("FromConfig does not make sense as case class", "2.0.1") - def productElement(x: Int) = x match { - case 0 ⇒ routerDispatcher - case _ ⇒ throw new IndexOutOfBoundsException(x.toString) - } - - @deprecated("FromConfig does not make sense as case class", "2.0.1") - def copy(d: String = Dispatchers.DefaultDispatcherId): FromConfig = new FromConfig(d) - - @deprecated("FromConfig does not make sense as case class", "2.0.1") - def canEqual(o: Any) = o.isInstanceOf[FromConfig] - - @deprecated("FromConfig does not make sense as case class", "2.0.1") - override def hashCode = ScalaRunTime._hashCode(this) - - @deprecated("FromConfig does not make sense as case class", "2.0.1") - override def toString = "FromConfig(" + routerDispatcher + ")" - - @deprecated("FromConfig does not make sense as case class", "2.0.1") - override def equals(other: Any): Boolean = other match { - case FromConfig(x) ⇒ x == routerDispatcher - case _ ⇒ false - } - } object RoundRobinRouter { @@ -510,9 +478,7 @@ case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] = * Constructor that sets nrOfInstances to be created. * Java API */ - def this(nr: Int) = { - this(nrOfInstances = nr) - } + def this(nr: Int) = this(nrOfInstances = nr) /** * Constructor that sets the routees to be used. @@ -520,9 +486,7 @@ case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] = * @param routeePaths string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ - def this(routeePaths: java.lang.Iterable[String]) = { - this(routees = iterableAsScalaIterable(routeePaths)) - } + def this(routeePaths: java.lang.Iterable[String]) = this(routees = iterableAsScalaIterable(routeePaths)) /** * Constructor that sets the resizer to be used. @@ -533,13 +497,13 @@ case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] = /** * Java API for setting routerDispatcher */ - def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId) + def withDispatcher(dispatcherId: String): RoundRobinRouter = copy(routerDispatcher = dispatcherId) /** * Java API for setting the supervisor strategy to be used for the “head” * Router actor. */ - def withSupervisorStrategy(strategy: SupervisorStrategy) = copy(supervisorStrategy = strategy) + def withSupervisorStrategy(strategy: SupervisorStrategy): RoundRobinRouter = copy(supervisorStrategy = strategy) } trait RoundRobinLike { this: RouterConfig ⇒ @@ -630,9 +594,7 @@ case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, * Constructor that sets nrOfInstances to be created. * Java API */ - def this(nr: Int) = { - this(nrOfInstances = nr) - } + def this(nr: Int) = this(nrOfInstances = nr) /** * Constructor that sets the routees to be used. @@ -640,9 +602,7 @@ case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, * @param routeePaths string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ - def this(routeePaths: java.lang.Iterable[String]) = { - this(routees = iterableAsScalaIterable(routeePaths)) - } + def this(routeePaths: java.lang.Iterable[String]) = this(routees = iterableAsScalaIterable(routeePaths)) /** * Constructor that sets the resizer to be used. @@ -653,13 +613,13 @@ case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, /** * Java API for setting routerDispatcher */ - def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId) + def withDispatcher(dispatcherId: String): RandomRouter = copy(routerDispatcher = dispatcherId) /** * Java API for setting the supervisor strategy to be used for the “head” * Router actor. */ - def withSupervisorStrategy(strategy: SupervisorStrategy) = copy(supervisorStrategy = strategy) + def withSupervisorStrategy(strategy: SupervisorStrategy): RandomRouter = copy(supervisorStrategy = strategy) } trait RandomLike { this: RouterConfig ⇒ @@ -756,9 +716,7 @@ case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[Strin * Constructor that sets nrOfInstances to be created. * Java API */ - def this(nr: Int) = { - this(nrOfInstances = nr) - } + def this(nr: Int) = this(nrOfInstances = nr) /** * Constructor that sets the routees to be used. @@ -766,9 +724,7 @@ case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[Strin * @param routeePaths string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ - def this(routeePaths: java.lang.Iterable[String]) = { - this(routees = iterableAsScalaIterable(routeePaths)) - } + def this(routeePaths: java.lang.Iterable[String]) = this(routees = iterableAsScalaIterable(routeePaths)) /** * Constructor that sets the resizer to be used. @@ -779,19 +735,16 @@ case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[Strin /** * Java API for setting routerDispatcher */ - def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId) + def withDispatcher(dispatcherId: String): SmallestMailboxRouter = copy(routerDispatcher = dispatcherId) /** * Java API for setting the supervisor strategy to be used for the “head” * Router actor. */ - def withSupervisorStrategy(strategy: SupervisorStrategy) = copy(supervisorStrategy = strategy) + def withSupervisorStrategy(strategy: SupervisorStrategy): SmallestMailboxRouter = copy(supervisorStrategy = strategy) } trait SmallestMailboxLike { this: RouterConfig ⇒ - - import java.security.SecureRandom - def nrOfInstances: Int def routees: Iterable[String] @@ -954,9 +907,7 @@ case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = N * Constructor that sets nrOfInstances to be created. * Java API */ - def this(nr: Int) = { - this(nrOfInstances = nr) - } + def this(nr: Int) = this(nrOfInstances = nr) /** * Constructor that sets the routees to be used. @@ -964,9 +915,7 @@ case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = N * @param routeePaths string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ - def this(routeePaths: java.lang.Iterable[String]) = { - this(routees = iterableAsScalaIterable(routeePaths)) - } + def this(routeePaths: java.lang.Iterable[String]) = this(routees = iterableAsScalaIterable(routeePaths)) /** * Constructor that sets the resizer to be used. @@ -977,13 +926,13 @@ case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = N /** * Java API for setting routerDispatcher */ - def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId) + def withDispatcher(dispatcherId: String): BroadcastRouter = copy(routerDispatcher = dispatcherId) /** * Java API for setting the supervisor strategy to be used for the “head” * Router actor. */ - def withSupervisorStrategy(strategy: SupervisorStrategy) = copy(supervisorStrategy = strategy) + def withSupervisorStrategy(strategy: SupervisorStrategy): BroadcastRouter = copy(supervisorStrategy = strategy) } trait BroadcastLike { this: RouterConfig ⇒ @@ -1069,9 +1018,7 @@ case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: It * Constructor that sets nrOfInstances to be created. * Java API */ - def this(nr: Int, w: Duration) = { - this(nrOfInstances = nr, within = w) - } + def this(nr: Int, w: Duration) = this(nrOfInstances = nr, within = w) /** * Constructor that sets the routees to be used. @@ -1079,9 +1026,8 @@ case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: It * @param routeePaths string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ - def this(routeePaths: java.lang.Iterable[String], w: Duration) = { + def this(routeePaths: java.lang.Iterable[String], w: Duration) = this(routees = iterableAsScalaIterable(routeePaths), within = w) - } /** * Constructor that sets the resizer to be used. @@ -1150,7 +1096,7 @@ trait Resizer { * This method is invoked only in the context of the Router actor in order to safely * create/stop children. */ - def resize(props: Props, routeeProvider: RouteeProvider) + def resize(props: Props, routeeProvider: RouteeProvider): Unit } case object DefaultResizer { @@ -1166,6 +1112,7 @@ case object DefaultResizer { messagesPerResize = resizerConfig.getInt("messages-per-resize")) } +//FIXME DOCUMENT ME case class DefaultResizer( /** * The fewest number of routees the router should ever have. @@ -1240,7 +1187,7 @@ case class DefaultResizer( def isTimeForResize(messageCounter: Long): Boolean = (messageCounter % messagesPerResize == 0) - def resize(props: Props, routeeProvider: RouteeProvider) { + def resize(props: Props, routeeProvider: RouteeProvider): Unit = { val currentRoutees = routeeProvider.routees val requestedCapacity = capacity(currentRoutees) @@ -1258,7 +1205,7 @@ case class DefaultResizer( * Give concurrent messages a chance to be placed in mailbox before * sending PoisonPill. */ - protected def delayedStop(scheduler: Scheduler, abandon: IndexedSeq[ActorRef]) { + protected def delayedStop(scheduler: Scheduler, abandon: IndexedSeq[ActorRef]): Unit = { if (abandon.nonEmpty) { if (stopDelay <= Duration.Zero) { abandon foreach (_ ! PoisonPill) @@ -1327,9 +1274,7 @@ case class DefaultResizer( * @param capacity current number of routees * @return proposed change in the capacity */ - def filter(pressure: Int, capacity: Int): Int = { - rampup(pressure, capacity) + backoff(pressure, capacity) - } + def filter(pressure: Int, capacity: Int): Int = rampup(pressure, capacity) + backoff(pressure, capacity) /** * Computes a proposed positive (or zero) capacity delta using diff --git a/akka-actor/src/main/scala/akka/serialization/Serialization.scala b/akka-actor/src/main/scala/akka/serialization/Serialization.scala index 03d03dc785..7355e4f7fb 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serialization.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serialization.scala @@ -14,8 +14,6 @@ import akka.util.NonFatal import scala.collection.mutable.ArrayBuffer import java.io.NotSerializableException -case class NoSerializerFoundException(m: String) extends AkkaException(m) - object Serialization { /** @@ -120,9 +118,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { possibilities(0)._2 } serializerMap.putIfAbsent(clazz, ser) match { - case null ⇒ - log.debug("Using serializer[{}] for message [{}]", ser.getClass.getName, clazz.getName) - ser + case null ⇒ log.debug("Using serializer[{}] for message [{}]", ser.getClass.getName, clazz.getName); ser case some ⇒ some } case ser ⇒ ser @@ -140,10 +136,8 @@ class Serialization(val system: ExtendedActorSystem) extends Extension { * A Map of serializer from alias to implementation (class implementing akka.serialization.Serializer) * By default always contains the following mapping: "java" -> akka.serialization.JavaSerializer */ - private val serializers: Map[String, Serializer] = { - for ((k: String, v: String) ← settings.Serializers) - yield k -> serializerOf(v).fold(throw _, identity) - } + private val serializers: Map[String, Serializer] = + for ((k: String, v: String) ← settings.Serializers) yield k -> serializerOf(v).fold(throw _, identity) /** * bindings is a Seq of tuple representing the mapping from Class to Serializer. diff --git a/akka-actor/src/main/scala/akka/serialization/Serializer.scala b/akka-actor/src/main/scala/akka/serialization/Serializer.scala index 5696201f62..f6300ca998 100644 --- a/akka-actor/src/main/scala/akka/serialization/Serializer.scala +++ b/akka-actor/src/main/scala/akka/serialization/Serializer.scala @@ -6,7 +6,6 @@ package akka.serialization import java.io.{ ObjectOutputStream, ByteArrayOutputStream, ObjectInputStream, ByteArrayInputStream } import akka.util.ClassLoaderObjectInputStream -import akka.actor.DynamicAccess import akka.actor.ExtendedActorSystem import scala.util.DynamicVariable diff --git a/akka-actor/src/main/scala/akka/util/BoundedBlockingQueue.scala b/akka-actor/src/main/scala/akka/util/BoundedBlockingQueue.scala index 7eb90b8ef0..c7c8308de0 100644 --- a/akka-actor/src/main/scala/akka/util/BoundedBlockingQueue.scala +++ b/akka-actor/src/main/scala/akka/util/BoundedBlockingQueue.scala @@ -8,6 +8,12 @@ import java.util.concurrent.locks.ReentrantLock import java.util.concurrent.{ TimeUnit, BlockingQueue } import java.util.{ AbstractQueue, Queue, Collection, Iterator } +/** + * BoundedBlockingQueue wraps any Queue and turns the result into a BlockingQueue with a limited capacity + * @param maxCapacity - the maximum capacity of this Queue, needs to be > 0 + * @param backing - the backing Queue + * @tparam E - The type of the contents of this Queue + */ class BoundedBlockingQueue[E <: AnyRef]( val maxCapacity: Int, private val backing: Queue[E]) extends AbstractQueue[E] with BlockingQueue[E] { @@ -22,7 +28,7 @@ class BoundedBlockingQueue[E <: AnyRef]( require(maxCapacity > 0) } - protected val lock = new ReentrantLock(false) + protected val lock = new ReentrantLock(false) // TODO might want to switch to ReentrantReadWriteLock private val notEmpty = lock.newCondition() private val notFull = lock.newCondition() diff --git a/akka-actor/src/main/scala/akka/util/ByteString.scala b/akka-actor/src/main/scala/akka/util/ByteString.scala index 6d869826a8..ac074d5b28 100644 --- a/akka-actor/src/main/scala/akka/util/ByteString.scala +++ b/akka-actor/src/main/scala/akka/util/ByteString.scala @@ -11,6 +11,7 @@ import scala.collection.mutable.{ Builder, WrappedArray } import scala.collection.immutable.{ IndexedSeq, VectorBuilder } import scala.collection.generic.CanBuildFrom +//FIXME MORE DOCS object ByteString { /** @@ -53,15 +54,16 @@ object ByteString { val empty: ByteString = CompactByteString(Array.empty[Byte]) - def newBuilder = new ByteStringBuilder + def newBuilder: ByteStringBuilder = new ByteStringBuilder - implicit def canBuildFrom = new CanBuildFrom[TraversableOnce[Byte], Byte, ByteString] { - def apply(from: TraversableOnce[Byte]) = newBuilder - def apply() = newBuilder - } + implicit val canBuildFrom: CanBuildFrom[TraversableOnce[Byte], Byte, ByteString] = + new CanBuildFrom[TraversableOnce[Byte], Byte, ByteString] { + def apply(ignore: TraversableOnce[Byte]): ByteStringBuilder = newBuilder + def apply(): ByteStringBuilder = newBuilder + } private[akka] object ByteString1C { - def apply(bytes: Array[Byte]) = new ByteString1C(bytes) + def apply(bytes: Array[Byte]): ByteString1C = new ByteString1C(bytes) } /** @@ -71,7 +73,7 @@ object ByteString { final class ByteString1C private (private val bytes: Array[Byte]) extends CompactByteString { def apply(idx: Int): Byte = bytes(idx) - override def length = bytes.length + override def length: Int = bytes.length def toArray: Array[Byte] = bytes.clone @@ -81,13 +83,11 @@ object ByteString { def compact: ByteString1C = this - def asByteBuffer: ByteBuffer = - toByteString1.asByteBuffer + def asByteBuffer: ByteBuffer = toByteString1.asByteBuffer def decodeString(charset: String): String = new String(bytes, charset) - def ++(that: ByteString): ByteString = - if (!that.isEmpty) toByteString1 ++ that else this + def ++(that: ByteString): ByteString = if (!that.isEmpty) toByteString1 ++ that else this override def slice(from: Int, until: Int): ByteString = if ((from != 0) || (until != length)) toByteString1.slice(from, until) @@ -96,12 +96,11 @@ object ByteString { override def copyToArray[A >: Byte](xs: Array[A], start: Int, len: Int): Unit = toByteString1.copyToArray(xs, start, len) - def copyToBuffer(buffer: ByteBuffer): Int = - toByteString1.copyToBuffer(buffer) + def copyToBuffer(buffer: ByteBuffer): Int = toByteString1.copyToBuffer(buffer) } private[akka] object ByteString1 { - def apply(bytes: Array[Byte]) = new ByteString1(bytes) + def apply(bytes: Array[Byte]): ByteString1 = new ByteString1(bytes) } /** @@ -113,7 +112,7 @@ object ByteString { def apply(idx: Int): Byte = bytes(checkRangeConvert(idx)) - private def checkRangeConvert(index: Int) = { + private def checkRangeConvert(index: Int): Int = { if (0 <= index && length > index) index + startIndex else @@ -128,8 +127,7 @@ object ByteString { override def clone: CompactByteString = ByteString1C(toArray) - def compact: CompactByteString = - if (length == bytes.length) ByteString1C(bytes) else clone + def compact: CompactByteString = if (length == bytes.length) ByteString1C(bytes) else clone def asByteBuffer: ByteBuffer = { val buffer = ByteBuffer.wrap(bytes, startIndex, length).asReadOnlyBuffer @@ -161,7 +159,6 @@ object ByteString { if (copyLength > 0) buffer.put(bytes, startIndex, copyLength) copyLength } - } private[akka] object ByteStrings { @@ -198,10 +195,11 @@ object ByteString { } // 0: both empty, 1: 2nd empty, 2: 1st empty, 3: neither empty + // Using length to check emptiness is prohibited by law def compare(b1: ByteString, b2: ByteString): Int = - if (b1.length == 0) - if (b2.length == 0) 0 else 2 - else if (b2.length == 0) 1 else 3 + if (b1.isEmpty) + if (b2.isEmpty) 0 else 2 + else if (b2.isEmpty) 1 else 3 } @@ -439,7 +437,7 @@ final class ByteStringBuilder extends Builder[Byte, ByteString] { private var _tempLength = 0 private var _tempCapacity = 0 - private def clearTemp() { + private def clearTemp(): Unit = { if (_tempLength > 0) { val arr = new Array[Byte](_tempLength) Array.copy(_temp, 0, arr, 0, _tempLength) @@ -448,14 +446,14 @@ final class ByteStringBuilder extends Builder[Byte, ByteString] { } } - private def resizeTemp(size: Int) { + private def resizeTemp(size: Int): Unit = { val newtemp = new Array[Byte](size) if (_tempLength > 0) Array.copy(_temp, 0, newtemp, 0, _tempLength) _temp = newtemp _tempCapacity = _temp.length } - private def ensureTempSize(size: Int) { + private def ensureTempSize(size: Int): Unit = { if (_tempCapacity < size || _tempCapacity == 0) { var newSize = if (_tempCapacity == 0) 16 else _tempCapacity * 2 while (newSize < size) newSize *= 2 @@ -498,7 +496,7 @@ final class ByteStringBuilder extends Builder[Byte, ByteString] { this } - def clear() { + def clear(): Unit = { _builder.clear _length = 0 _tempLength = 0 diff --git a/akka-actor/src/main/scala/akka/util/ClassLoaderObjectInputStream.scala b/akka-actor/src/main/scala/akka/util/ClassLoaderObjectInputStream.scala index 3ad55d69eb..ab2514861e 100644 --- a/akka-actor/src/main/scala/akka/util/ClassLoaderObjectInputStream.scala +++ b/akka-actor/src/main/scala/akka/util/ClassLoaderObjectInputStream.scala @@ -6,6 +6,13 @@ package akka.util import java.io.{ InputStream, ObjectInputStream, ObjectStreamClass } +/** + * ClassLoaderObjectInputStream tries to utilize the provided ClassLoader to load Classes and falls + * back to ObjectInputStreams resolver. + * + * @param classLoader - the ClassLoader which is to be used primarily + * @param is - the InputStream that is wrapped + */ class ClassLoaderObjectInputStream(classLoader: ClassLoader, is: InputStream) extends ObjectInputStream(is) { override protected def resolveClass(objectStreamClass: ObjectStreamClass): Class[_] = try Class.forName(objectStreamClass.getName, false, classLoader) catch { diff --git a/akka-actor/src/main/scala/akka/util/Convert.scala b/akka-actor/src/main/scala/akka/util/Convert.scala index a805b17fb2..3fead7aef7 100644 --- a/akka-actor/src/main/scala/akka/util/Convert.scala +++ b/akka-actor/src/main/scala/akka/util/Convert.scala @@ -3,7 +3,7 @@ */ package akka.util - +//FIXME DOCS! object Convert { def intToBytes(value: Int): Array[Byte] = { diff --git a/akka-actor/src/main/scala/akka/util/Crypt.scala b/akka-actor/src/main/scala/akka/util/Crypt.scala index 7dd678e748..280cd90768 100644 --- a/akka-actor/src/main/scala/akka/util/Crypt.scala +++ b/akka-actor/src/main/scala/akka/util/Crypt.scala @@ -5,7 +5,7 @@ package akka.util import java.security.{ MessageDigest, SecureRandom } - +//FIXME DOCS object Crypt { val hex = "0123456789ABCDEF" val lineSeparator = System.getProperty("line.separator") @@ -32,7 +32,7 @@ object Crypt { } def hexify(bytes: Array[Byte]): String = { - val builder = new StringBuilder + val builder = new StringBuilder(bytes.length * 2) bytes.foreach { byte ⇒ builder.append(hex.charAt((byte & 0xF0) >> 4)).append(hex.charAt(byte & 0xF)) } builder.toString } diff --git a/akka-actor/src/main/scala/akka/util/Duration.scala b/akka-actor/src/main/scala/akka/util/Duration.scala index a213fe1869..b37cf24c3b 100644 --- a/akka-actor/src/main/scala/akka/util/Duration.scala +++ b/akka-actor/src/main/scala/akka/util/Duration.scala @@ -110,6 +110,7 @@ object Duration { } val Zero: FiniteDuration = new FiniteDuration(0, NANOSECONDS) + val Undefined: Duration = new Duration with Infinite { override def toString = "Duration.Undefined" override def equals(other: Any) = other.asInstanceOf[AnyRef] eq this @@ -166,8 +167,8 @@ object Duration { * including itself. */ val Inf: Duration = new Duration with Infinite { - override def toString = "Duration.Inf" - def compare(other: Duration) = if (other eq this) 0 else 1 + override def toString: String = "Duration.Inf" + def compare(other: Duration): Int = if (other eq this) 0 else 1 def unary_- : Duration = MinusInf } @@ -177,7 +178,7 @@ object Duration { */ val MinusInf: Duration = new Duration with Infinite { override def toString = "Duration.MinusInf" - def compare(other: Duration) = if (other eq this) 0 else -1 + def compare(other: Duration): Int = if (other eq this) 0 else -1 def unary_- : Duration = Inf } @@ -188,7 +189,7 @@ object Duration { def parse(s: String): Duration = unapply(s).get implicit object DurationIsOrdered extends Ordering[Duration] { - def compare(a: Duration, b: Duration) = a compare b + def compare(a: Duration, b: Duration): Int = a compare b } } @@ -263,17 +264,17 @@ abstract class Duration extends Serializable with Ordered[Duration] { def fromNow: Deadline = Deadline.now + this // Java API - def lt(other: Duration) = this < other - def lteq(other: Duration) = this <= other - def gt(other: Duration) = this > other - def gteq(other: Duration) = this >= other - def plus(other: Duration) = this + other - def minus(other: Duration) = this - other - def mul(factor: Double) = this * factor - def div(factor: Double) = this / factor - def div(other: Duration) = this / other - def neg() = -this - def isFinite() = finite_? + def lt(other: Duration): Boolean = this < other + def lteq(other: Duration): Boolean = this <= other + def gt(other: Duration): Boolean = this > other + def gteq(other: Duration): Boolean = this >= other + def plus(other: Duration): Duration = this + other + def minus(other: Duration): Duration = this - other + def mul(factor: Double): Duration = this * factor + def div(factor: Double): Duration = this / factor + def div(other: Duration): Double = this / other + def neg(): Duration = -this + def isFinite(): Boolean = finite_? } object FiniteDuration { @@ -349,31 +350,19 @@ class FiniteDuration(val length: Long, val unit: TimeUnit) extends Duration { else c } - def +(other: Duration) = { - if (!other.finite_?) { - other - } else { - fromNanos(add(toNanos, other.toNanos)) - } - } + def +(other: Duration): Duration = if (!other.finite_?) other else fromNanos(add(toNanos, other.toNanos)) - def -(other: Duration) = { - if (!other.finite_?) { - other - } else { - fromNanos(add(toNanos, -other.toNanos)) - } - } + def -(other: Duration): Duration = if (!other.finite_?) other else fromNanos(add(toNanos, -other.toNanos)) - def *(factor: Double) = fromNanos(long2double(toNanos) * factor) + def *(factor: Double): FiniteDuration = fromNanos(long2double(toNanos) * factor) - def /(factor: Double) = fromNanos(long2double(toNanos) / factor) + def /(factor: Double): FiniteDuration = fromNanos(long2double(toNanos) / factor) - def /(other: Duration) = if (other.finite_?) long2double(toNanos) / other.toNanos else 0 + def /(other: Duration): Double = if (other.finite_?) long2double(toNanos) / other.toNanos else 0 - def unary_- = Duration(-length, unit) + def unary_- : FiniteDuration = Duration(-length, unit) - def finite_? = true + def finite_? : Boolean = true override def equals(other: Any) = (other.asInstanceOf[AnyRef] eq this) || other.isInstanceOf[FiniteDuration] && @@ -385,178 +374,74 @@ class FiniteDuration(val length: Long, val unit: TimeUnit) extends Duration { } } -class DurationInt(n: Int) { +private[akka] trait DurationOps { import duration.Classifier + protected def from(timeUnit: TimeUnit): FiniteDuration + def nanoseconds: FiniteDuration = from(NANOSECONDS) + def nanos: FiniteDuration = from(NANOSECONDS) + def nanosecond: FiniteDuration = from(NANOSECONDS) + def nano: FiniteDuration = from(NANOSECONDS) - def nanoseconds = Duration(n, NANOSECONDS) - def nanos = Duration(n, NANOSECONDS) - def nanosecond = Duration(n, NANOSECONDS) - def nano = Duration(n, NANOSECONDS) + def microseconds: FiniteDuration = from(MICROSECONDS) + def micros: FiniteDuration = from(MICROSECONDS) + def microsecond: FiniteDuration = from(MICROSECONDS) + def micro: FiniteDuration = from(MICROSECONDS) - def microseconds = Duration(n, MICROSECONDS) - def micros = Duration(n, MICROSECONDS) - def microsecond = Duration(n, MICROSECONDS) - def micro = Duration(n, MICROSECONDS) + def milliseconds: FiniteDuration = from(MILLISECONDS) + def millis: FiniteDuration = from(MILLISECONDS) + def millisecond: FiniteDuration = from(MILLISECONDS) + def milli: FiniteDuration = from(MILLISECONDS) - def milliseconds = Duration(n, MILLISECONDS) - def millis = Duration(n, MILLISECONDS) - def millisecond = Duration(n, MILLISECONDS) - def milli = Duration(n, MILLISECONDS) + def seconds: FiniteDuration = from(SECONDS) + def second: FiniteDuration = from(SECONDS) - def seconds = Duration(n, SECONDS) - def second = Duration(n, SECONDS) + def minutes: FiniteDuration = from(MINUTES) + def minute: FiniteDuration = from(MINUTES) - def minutes = Duration(n, MINUTES) - def minute = Duration(n, MINUTES) + def hours: FiniteDuration = from(HOURS) + def hour: FiniteDuration = from(HOURS) - def hours = Duration(n, HOURS) - def hour = Duration(n, HOURS) + def days: FiniteDuration = from(DAYS) + def day: FiniteDuration = from(DAYS) - def days = Duration(n, DAYS) - def day = Duration(n, DAYS) + def nanoseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(NANOSECONDS)) + def nanos[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(NANOSECONDS)) + def nanosecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(NANOSECONDS)) + def nano[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(NANOSECONDS)) - def nanoseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS)) - def nanos[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS)) - def nanosecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS)) - def nano[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS)) + def microseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MICROSECONDS)) + def micros[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MICROSECONDS)) + def microsecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MICROSECONDS)) + def micro[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MICROSECONDS)) - def microseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS)) - def micros[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS)) - def microsecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS)) - def micro[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS)) + def milliseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MILLISECONDS)) + def millis[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MILLISECONDS)) + def millisecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MILLISECONDS)) + def milli[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MILLISECONDS)) - def milliseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS)) - def millis[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS)) - def millisecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS)) - def milli[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS)) + def seconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(SECONDS)) + def second[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(SECONDS)) - def seconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, SECONDS)) - def second[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, SECONDS)) + def minutes[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MINUTES)) + def minute[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MINUTES)) - def minutes[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MINUTES)) - def minute[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MINUTES)) + def hours[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(HOURS)) + def hour[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(HOURS)) - def hours[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, HOURS)) - def hour[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, HOURS)) - - def days[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, DAYS)) - def day[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, DAYS)) + def days[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(DAYS)) + def day[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(DAYS)) } -class DurationLong(n: Long) { - import duration.Classifier - - def nanoseconds = Duration(n, NANOSECONDS) - def nanos = Duration(n, NANOSECONDS) - def nanosecond = Duration(n, NANOSECONDS) - def nano = Duration(n, NANOSECONDS) - - def microseconds = Duration(n, MICROSECONDS) - def micros = Duration(n, MICROSECONDS) - def microsecond = Duration(n, MICROSECONDS) - def micro = Duration(n, MICROSECONDS) - - def milliseconds = Duration(n, MILLISECONDS) - def millis = Duration(n, MILLISECONDS) - def millisecond = Duration(n, MILLISECONDS) - def milli = Duration(n, MILLISECONDS) - - def seconds = Duration(n, SECONDS) - def second = Duration(n, SECONDS) - - def minutes = Duration(n, MINUTES) - def minute = Duration(n, MINUTES) - - def hours = Duration(n, HOURS) - def hour = Duration(n, HOURS) - - def days = Duration(n, DAYS) - def day = Duration(n, DAYS) - - def nanoseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS)) - def nanos[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS)) - def nanosecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS)) - def nano[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS)) - - def microseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS)) - def micros[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS)) - def microsecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS)) - def micro[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS)) - - def milliseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS)) - def millis[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS)) - def millisecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS)) - def milli[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS)) - - def seconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, SECONDS)) - def second[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, SECONDS)) - - def minutes[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MINUTES)) - def minute[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MINUTES)) - - def hours[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, HOURS)) - def hour[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, HOURS)) - - def days[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, DAYS)) - def day[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, DAYS)) +class DurationInt(n: Int) extends DurationOps { + override protected def from(timeUnit: TimeUnit): FiniteDuration = Duration(n, timeUnit) } -class DurationDouble(d: Double) { - import duration.Classifier +class DurationLong(n: Long) extends DurationOps { + override protected def from(timeUnit: TimeUnit): FiniteDuration = Duration(n, timeUnit) +} - def nanoseconds = Duration(d, NANOSECONDS) - def nanos = Duration(d, NANOSECONDS) - def nanosecond = Duration(d, NANOSECONDS) - def nano = Duration(d, NANOSECONDS) - - def microseconds = Duration(d, MICROSECONDS) - def micros = Duration(d, MICROSECONDS) - def microsecond = Duration(d, MICROSECONDS) - def micro = Duration(d, MICROSECONDS) - - def milliseconds = Duration(d, MILLISECONDS) - def millis = Duration(d, MILLISECONDS) - def millisecond = Duration(d, MILLISECONDS) - def milli = Duration(d, MILLISECONDS) - - def seconds = Duration(d, SECONDS) - def second = Duration(d, SECONDS) - - def minutes = Duration(d, MINUTES) - def minute = Duration(d, MINUTES) - - def hours = Duration(d, HOURS) - def hour = Duration(d, HOURS) - - def days = Duration(d, DAYS) - def day = Duration(d, DAYS) - - def nanoseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, NANOSECONDS)) - def nanos[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, NANOSECONDS)) - def nanosecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, NANOSECONDS)) - def nano[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, NANOSECONDS)) - - def microseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MICROSECONDS)) - def micros[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MICROSECONDS)) - def microsecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MICROSECONDS)) - def micro[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MICROSECONDS)) - - def milliseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MILLISECONDS)) - def millis[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MILLISECONDS)) - def millisecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MILLISECONDS)) - def milli[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MILLISECONDS)) - - def seconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, SECONDS)) - def second[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, SECONDS)) - - def minutes[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MINUTES)) - def minute[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, MINUTES)) - - def hours[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, HOURS)) - def hour[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, HOURS)) - - def days[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, DAYS)) - def day[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(d, DAYS)) +class DurationDouble(d: Double) extends DurationOps { + override protected def from(timeUnit: TimeUnit): FiniteDuration = Duration(d, timeUnit) } //TODO add @SerialVersionUID(1L) when SI-4804 is fixed @@ -565,24 +450,27 @@ case class Timeout(duration: Duration) { def this(length: Long, unit: TimeUnit) = this(Duration(length, unit)) } +/** + * A Timeout is a wrapper on top of Duration to be more specific about what the duration means. + */ object Timeout { /** * A timeout with zero duration, will cause most requests to always timeout. */ - val zero = new Timeout(Duration.Zero) + val zero: Timeout = new Timeout(Duration.Zero) /** * A Timeout with infinite duration. Will never timeout. Use extreme caution with this * as it may cause memory leaks, blocked threads, or may not even be supported by * the receiver, which would result in an exception. */ - val never = new Timeout(Duration.Inf) + val never: Timeout = new Timeout(Duration.Inf) - def apply(timeout: Long) = new Timeout(timeout) - def apply(length: Long, unit: TimeUnit) = new Timeout(length, unit) + def apply(timeout: Long): Timeout = new Timeout(timeout) + def apply(length: Long, unit: TimeUnit): Timeout = new Timeout(length, unit) - implicit def durationToTimeout(duration: Duration) = new Timeout(duration) - implicit def intToTimeout(timeout: Int) = new Timeout(timeout) - implicit def longToTimeout(timeout: Long) = new Timeout(timeout) + implicit def durationToTimeout(duration: Duration): Timeout = new Timeout(duration) + implicit def intToTimeout(timeout: Int): Timeout = new Timeout(timeout) + implicit def longToTimeout(timeout: Long): Timeout = new Timeout(timeout) } diff --git a/akka-actor/src/main/scala/akka/util/Helpers.scala b/akka-actor/src/main/scala/akka/util/Helpers.scala index 25cb279f2e..a3618359ac 100644 --- a/akka-actor/src/main/scala/akka/util/Helpers.scala +++ b/akka-actor/src/main/scala/akka/util/Helpers.scala @@ -45,18 +45,13 @@ object Helpers { else base64(next, sb) } - def ignore[E: Manifest](body: ⇒ Unit) { - try { - body - } catch { - case e if manifest[E].erasure.isAssignableFrom(e.getClass) ⇒ () - } - } + //FIXME docs + def ignore[E: Manifest](body: ⇒ Unit): Unit = + try body catch { case e if manifest[E].erasure.isAssignableFrom(e.getClass) ⇒ () } - def withPrintStackTraceOnError(body: ⇒ Unit) { - try { - body - } catch { + //FIXME docs + def withPrintStackTraceOnError(body: ⇒ Unit): Unit = { + try body catch { case e: Throwable ⇒ val sw = new java.io.StringWriter() var root = e diff --git a/akka-actor/src/main/scala/akka/util/Index.scala b/akka-actor/src/main/scala/akka/util/Index.scala index 1153c9e045..3289ed8f13 100644 --- a/akka-actor/src/main/scala/akka/util/Index.scala +++ b/akka-actor/src/main/scala/akka/util/Index.scala @@ -91,7 +91,7 @@ class Index[K, V](val mapSize: Int, val valueComparator: Comparator[V]) { /** * Applies the supplied function to all keys and their values */ - def foreach(fun: (K, V) ⇒ Unit) { + def foreach(fun: (K, V) ⇒ Unit): Unit = { import scala.collection.JavaConversions._ container.entrySet foreach { e ⇒ e.getValue.foreach(fun(e.getKey, _)) } } @@ -112,7 +112,7 @@ class Index[K, V](val mapSize: Int, val valueComparator: Comparator[V]) { /** * Returns the key set. */ - def keys = scala.collection.JavaConversions.collectionAsScalaIterable(container.keySet) + def keys: Iterable[K] = scala.collection.JavaConversions.collectionAsScalaIterable(container.keySet) /** * Disassociates the value of type V from the key of type K diff --git a/akka-actor/src/main/scala/akka/util/LockUtil.scala b/akka-actor/src/main/scala/akka/util/LockUtil.scala index 14c787d3f6..da93170019 100644 --- a/akka-actor/src/main/scala/akka/util/LockUtil.scala +++ b/akka-actor/src/main/scala/akka/util/LockUtil.scala @@ -24,9 +24,7 @@ class Switch(startAsOn: Boolean = false) { protected def transcend(from: Boolean, action: ⇒ Unit): Boolean = synchronized { if (switch.compareAndSet(from, !from)) { - try { - action - } catch { + try action catch { case e ⇒ switch.compareAndSet(!from, from) // revert status throw e @@ -62,18 +60,12 @@ class Switch(startAsOn: Boolean = false) { /** * Executes the provided action and returns its value if the switch is IMMEDIATELY on (i.e. no lock involved) */ - def ifOnYield[T](action: ⇒ T): Option[T] = { - if (switch.get) Some(action) - else None - } + def ifOnYield[T](action: ⇒ T): Option[T] = if (switch.get) Some(action) else None /** * Executes the provided action and returns its value if the switch is IMMEDIATELY off (i.e. no lock involved) */ - def ifOffYield[T](action: ⇒ T): Option[T] = { - if (!switch.get) Some(action) - else None - } + def ifOffYield[T](action: ⇒ T): Option[T] = if (!switch.get) Some(action) else None /** * Executes the provided action and returns if the action was executed or not, if the switch is IMMEDIATELY on (i.e. no lock involved) @@ -138,15 +130,15 @@ class Switch(startAsOn: Boolean = false) { /** * Executes the given code while holding this switch’s lock, i.e. protected from concurrent modification of the switch status. */ - def locked[T](code: ⇒ T) = synchronized { code } + def locked[T](code: ⇒ T): T = synchronized { code } /** * Returns whether the switch is IMMEDIATELY on (no locking) */ - def isOn = switch.get + def isOn: Boolean = switch.get /** * Returns whether the switch is IMMEDDIATELY off (no locking) */ - def isOff = !isOn + def isOff: Boolean = !isOn } diff --git a/akka-actor/src/main/scala/akka/util/Reflect.scala b/akka-actor/src/main/scala/akka/util/Reflect.scala index 25c56a983f..3a46edeab1 100644 --- a/akka-actor/src/main/scala/akka/util/Reflect.scala +++ b/akka-actor/src/main/scala/akka/util/Reflect.scala @@ -6,8 +6,10 @@ package akka.util /** * Collection of internal reflection utilities which may or may not be * available (most services specific to HotSpot, but fails gracefully). + * + * INTERNAL API */ -object Reflect { +private[akka] object Reflect { /** * This optionally holds a function which looks N levels above itself diff --git a/akka-actor/src/main/scala/akka/util/Unsafe.java b/akka-actor/src/main/scala/akka/util/Unsafe.java index 608cb3d46e..ace3c1baac 100644 --- a/akka-actor/src/main/scala/akka/util/Unsafe.java +++ b/akka-actor/src/main/scala/akka/util/Unsafe.java @@ -7,6 +7,9 @@ package akka.util; import java.lang.reflect.Field; +/** + * INTERNAL API + */ public final class Unsafe { public final static sun.misc.Unsafe instance; static { diff --git a/akka-actor/src/main/scala/akka/util/cps/package.scala b/akka-actor/src/main/scala/akka/util/cps/package.scala index 198c2beacd..a1b4bc39eb 100644 --- a/akka-actor/src/main/scala/akka/util/cps/package.scala +++ b/akka-actor/src/main/scala/akka/util/cps/package.scala @@ -7,6 +7,7 @@ package akka.util import scala.util.continuations._ import akka.dispatch.MessageDispatcher +//FIXME Needs docs package object cps { def matchC[A, B, C, D](in: A)(pf: PartialFunction[A, B @cpsParam[C, D]]): B @cpsParam[C, D] = pf(in) diff --git a/akka-actor/src/main/scala/akka/util/duration/package.scala b/akka-actor/src/main/scala/akka/util/duration/package.scala index 7f14a0be48..6a7d28a6e6 100644 --- a/akka-actor/src/main/scala/akka/util/duration/package.scala +++ b/akka-actor/src/main/scala/akka/util/duration/package.scala @@ -5,7 +5,7 @@ package akka.util import java.util.concurrent.TimeUnit - +//FIXME Needs docs package object duration { trait Classifier[C] { type R @@ -15,38 +15,32 @@ package object duration { object span implicit object spanConvert extends Classifier[span.type] { type R = FiniteDuration - def convert(d: FiniteDuration) = d + def convert(d: FiniteDuration): FiniteDuration = d } object fromNow implicit object fromNowConvert extends Classifier[fromNow.type] { type R = Deadline - def convert(d: FiniteDuration) = Deadline.now + d + def convert(d: FiniteDuration): Deadline = Deadline.now + d } - implicit def intToDurationInt(n: Int) = new DurationInt(n) - implicit def longToDurationLong(n: Long) = new DurationLong(n) - implicit def doubleToDurationDouble(d: Double) = new DurationDouble(d) + implicit def intToDurationInt(n: Int): DurationInt = new DurationInt(n) + implicit def longToDurationLong(n: Long): DurationLong = new DurationLong(n) + implicit def doubleToDurationDouble(d: Double): DurationDouble = new DurationDouble(d) - implicit def pairIntToDuration(p: (Int, TimeUnit)) = Duration(p._1, p._2) - implicit def pairLongToDuration(p: (Long, TimeUnit)) = Duration(p._1, p._2) - implicit def durationToPair(d: Duration) = (d.length, d.unit) + implicit def pairIntToDuration(p: (Int, TimeUnit)): FiniteDuration = Duration(p._1, p._2) + implicit def pairLongToDuration(p: (Long, TimeUnit)): FiniteDuration = Duration(p._1, p._2) + implicit def durationToPair(d: Duration): (Long, TimeUnit) = (d.length, d.unit) /* * avoid reflection based invocation by using non-duck type */ - class IntMult(i: Int) { - def *(d: Duration) = d * i - } - implicit def intMult(i: Int) = new IntMult(i) + class IntMult(i: Int) { def *(d: Duration): Duration = d * i } + implicit def intMult(i: Int): IntMult = new IntMult(i) - class LongMult(l: Long) { - def *(d: Duration) = d * l - } - implicit def longMult(l: Long) = new LongMult(l) + class LongMult(l: Long) { def *(d: Duration): Duration = d * l } + implicit def longMult(l: Long): LongMult = new LongMult(l) - class DoubleMult(f: Double) { - def *(d: Duration) = d * f - } - implicit def doubleMult(f: Double) = new DoubleMult(f) + class DoubleMult(f: Double) { def *(d: Duration): Duration = d * f } + implicit def doubleMult(f: Double): DoubleMult = new DoubleMult(f) }