Huge refactor + preparing for binary compatibility, last stretch for akka-actor.jar...

This commit is contained in:
Viktor Klang 2012-05-18 19:25:43 +02:00
parent 134f1a19a5
commit 94e71b7a18
18 changed files with 218 additions and 408 deletions

View file

@ -5,19 +5,26 @@
package akka package akka
object AkkaException { object AkkaException {
//FIXME DOC
def toStringWithStackTrace(throwable: Throwable): String = throwable match { def toStringWithStackTrace(throwable: Throwable): String = throwable match {
case null "Unknown Throwable: was 'null'" case null "Unknown Throwable: was 'null'"
case ae: AkkaException ae.toLongString case ae: AkkaException ae.toLongString
case e "%s:%s\n%s" format (e.getClass.getName, e.getMessage, stackTraceToString(e)) case e "%s:%s\n%s" format (e.getClass.getName, e.getMessage, stackTraceToString(e))
} }
def stackTraceToString(throwable: Throwable): String = { /**
val trace = throwable.getStackTrace * Returns the given Throwables stack trace as a String, or the empty String if no trace is found
val sb = new StringBuilder * @param throwable
for (i 0 until trace.length) * @return
sb.append("\tat %s\n" format trace(i)) */
sb.toString 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 //TODO add @SerialVersionUID(1L) when SI-4804 is fixed
class AkkaException(message: String = "", cause: Throwable = null) extends RuntimeException(message, cause) with Serializable { 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 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)
} }
/** /**

View file

@ -1,19 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
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

View file

@ -47,12 +47,11 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup
ref: InternalActorRef, ref: InternalActorRef,
props: Props, props: Props,
supervisor: InternalActorRef, supervisor: InternalActorRef,
receiveTimeout: Option[Duration]): ActorCell = receiveTimeout: Option[Duration]): ActorCell = {
{ val cell = super.newActorCell(system, ref, props, supervisor, receiveTimeout)
val cell = super.newActorCell(system, ref, props, supervisor, receiveTimeout) Unsafe.instance.monitorEnter(cell)
Unsafe.instance.monitorEnter(cell) cell
cell }
}
private[akka] val routerConfig = _props.routerConfig private[akka] val routerConfig = _props.routerConfig
private[akka] val routeeProps = _props.copy(routerConfig = NoRouter) private[akka] val routeeProps = _props.copy(routerConfig = NoRouter)
@ -303,8 +302,8 @@ trait Router extends Actor {
final def receive = ({ final def receive = ({
case Router.Resize case Router.Resize
try ref.routerConfig.resizer foreach (_.resize(ref.routeeProps, ref.routeeProvider)) val ab = ref.resizeInProgress
finally assert(ref.resizeInProgress.getAndSet(false)) if (ab.get) try ref.routerConfig.resizer foreach (_.resize(ref.routeeProps, ref.routeeProvider)) finally ab.set(false)
case Terminated(child) case Terminated(child)
ref.removeRoutees(IndexedSeq(child)) ref.removeRoutees(IndexedSeq(child))
@ -319,6 +318,9 @@ trait Router extends Actor {
} }
} }
/**
* INTERNAL API
*/
private object Router { private object Router {
case object Resize case object Resize
@ -372,9 +374,9 @@ case class Destination(sender: ActorRef, recipient: ActorRef)
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed //TODO add @SerialVersionUID(1L) when SI-4804 is fixed
abstract class NoRouter extends RouterConfig abstract class NoRouter extends RouterConfig
case object NoRouter extends NoRouter { 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 routerDispatcher: String = ""
def supervisorStrategy = null def supervisorStrategy = null // FIXME null, really??
override def withFallback(other: RouterConfig): RouterConfig = other 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 //TODO add @SerialVersionUID(1L) when SI-4804 is fixed
class FromConfig(val routerDispatcher: String = Dispatchers.DefaultDispatcherId) class FromConfig(val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
extends RouterConfig extends RouterConfig
with Product with Serializable {
with Serializable
with Equals {
def this() = this(Dispatchers.DefaultDispatcherId) 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)") throw new ConfigurationException("router " + routeeProvider.context.self + " needs external configuration from file (e.g. application.conf)")
def supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy 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 { object RoundRobinRouter {
@ -510,9 +478,7 @@ case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] =
* Constructor that sets nrOfInstances to be created. * Constructor that sets nrOfInstances to be created.
* Java API * Java API
*/ */
def this(nr: Int) = { def this(nr: Int) = this(nrOfInstances = nr)
this(nrOfInstances = nr)
}
/** /**
* Constructor that sets the routees to be used. * 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 * @param routeePaths string representation of the actor paths of the routees that will be looked up
* using `actorFor` in [[akka.actor.ActorRefProvider]] * using `actorFor` in [[akka.actor.ActorRefProvider]]
*/ */
def this(routeePaths: java.lang.Iterable[String]) = { def this(routeePaths: java.lang.Iterable[String]) = this(routees = iterableAsScalaIterable(routeePaths))
this(routees = iterableAsScalaIterable(routeePaths))
}
/** /**
* Constructor that sets the resizer to be used. * 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 * 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 * Java API for setting the supervisor strategy to be used for the head
* Router actor. * Router actor.
*/ */
def withSupervisorStrategy(strategy: SupervisorStrategy) = copy(supervisorStrategy = strategy) def withSupervisorStrategy(strategy: SupervisorStrategy): RoundRobinRouter = copy(supervisorStrategy = strategy)
} }
trait RoundRobinLike { this: RouterConfig 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. * Constructor that sets nrOfInstances to be created.
* Java API * Java API
*/ */
def this(nr: Int) = { def this(nr: Int) = this(nrOfInstances = nr)
this(nrOfInstances = nr)
}
/** /**
* Constructor that sets the routees to be used. * 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 * @param routeePaths string representation of the actor paths of the routees that will be looked up
* using `actorFor` in [[akka.actor.ActorRefProvider]] * using `actorFor` in [[akka.actor.ActorRefProvider]]
*/ */
def this(routeePaths: java.lang.Iterable[String]) = { def this(routeePaths: java.lang.Iterable[String]) = this(routees = iterableAsScalaIterable(routeePaths))
this(routees = iterableAsScalaIterable(routeePaths))
}
/** /**
* Constructor that sets the resizer to be used. * 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 * 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 * Java API for setting the supervisor strategy to be used for the head
* Router actor. * Router actor.
*/ */
def withSupervisorStrategy(strategy: SupervisorStrategy) = copy(supervisorStrategy = strategy) def withSupervisorStrategy(strategy: SupervisorStrategy): RandomRouter = copy(supervisorStrategy = strategy)
} }
trait RandomLike { this: RouterConfig trait RandomLike { this: RouterConfig
@ -756,9 +716,7 @@ case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[Strin
* Constructor that sets nrOfInstances to be created. * Constructor that sets nrOfInstances to be created.
* Java API * Java API
*/ */
def this(nr: Int) = { def this(nr: Int) = this(nrOfInstances = nr)
this(nrOfInstances = nr)
}
/** /**
* Constructor that sets the routees to be used. * 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 * @param routeePaths string representation of the actor paths of the routees that will be looked up
* using `actorFor` in [[akka.actor.ActorRefProvider]] * using `actorFor` in [[akka.actor.ActorRefProvider]]
*/ */
def this(routeePaths: java.lang.Iterable[String]) = { def this(routeePaths: java.lang.Iterable[String]) = this(routees = iterableAsScalaIterable(routeePaths))
this(routees = iterableAsScalaIterable(routeePaths))
}
/** /**
* Constructor that sets the resizer to be used. * 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 * 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 * Java API for setting the supervisor strategy to be used for the head
* Router actor. * Router actor.
*/ */
def withSupervisorStrategy(strategy: SupervisorStrategy) = copy(supervisorStrategy = strategy) def withSupervisorStrategy(strategy: SupervisorStrategy): SmallestMailboxRouter = copy(supervisorStrategy = strategy)
} }
trait SmallestMailboxLike { this: RouterConfig trait SmallestMailboxLike { this: RouterConfig
import java.security.SecureRandom
def nrOfInstances: Int def nrOfInstances: Int
def routees: Iterable[String] 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. * Constructor that sets nrOfInstances to be created.
* Java API * Java API
*/ */
def this(nr: Int) = { def this(nr: Int) = this(nrOfInstances = nr)
this(nrOfInstances = nr)
}
/** /**
* Constructor that sets the routees to be used. * 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 * @param routeePaths string representation of the actor paths of the routees that will be looked up
* using `actorFor` in [[akka.actor.ActorRefProvider]] * using `actorFor` in [[akka.actor.ActorRefProvider]]
*/ */
def this(routeePaths: java.lang.Iterable[String]) = { def this(routeePaths: java.lang.Iterable[String]) = this(routees = iterableAsScalaIterable(routeePaths))
this(routees = iterableAsScalaIterable(routeePaths))
}
/** /**
* Constructor that sets the resizer to be used. * 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 * 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 * Java API for setting the supervisor strategy to be used for the head
* Router actor. * Router actor.
*/ */
def withSupervisorStrategy(strategy: SupervisorStrategy) = copy(supervisorStrategy = strategy) def withSupervisorStrategy(strategy: SupervisorStrategy): BroadcastRouter = copy(supervisorStrategy = strategy)
} }
trait BroadcastLike { this: RouterConfig trait BroadcastLike { this: RouterConfig
@ -1069,9 +1018,7 @@ case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: It
* Constructor that sets nrOfInstances to be created. * Constructor that sets nrOfInstances to be created.
* Java API * Java API
*/ */
def this(nr: Int, w: Duration) = { def this(nr: Int, w: Duration) = this(nrOfInstances = nr, within = w)
this(nrOfInstances = nr, within = w)
}
/** /**
* Constructor that sets the routees to be used. * 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 * @param routeePaths string representation of the actor paths of the routees that will be looked up
* using `actorFor` in [[akka.actor.ActorRefProvider]] * 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) this(routees = iterableAsScalaIterable(routeePaths), within = w)
}
/** /**
* Constructor that sets the resizer to be used. * 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 * This method is invoked only in the context of the Router actor in order to safely
* create/stop children. * create/stop children.
*/ */
def resize(props: Props, routeeProvider: RouteeProvider) def resize(props: Props, routeeProvider: RouteeProvider): Unit
} }
case object DefaultResizer { case object DefaultResizer {
@ -1166,6 +1112,7 @@ case object DefaultResizer {
messagesPerResize = resizerConfig.getInt("messages-per-resize")) messagesPerResize = resizerConfig.getInt("messages-per-resize"))
} }
//FIXME DOCUMENT ME
case class DefaultResizer( case class DefaultResizer(
/** /**
* The fewest number of routees the router should ever have. * 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 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 currentRoutees = routeeProvider.routees
val requestedCapacity = capacity(currentRoutees) val requestedCapacity = capacity(currentRoutees)
@ -1258,7 +1205,7 @@ case class DefaultResizer(
* Give concurrent messages a chance to be placed in mailbox before * Give concurrent messages a chance to be placed in mailbox before
* sending PoisonPill. * sending PoisonPill.
*/ */
protected def delayedStop(scheduler: Scheduler, abandon: IndexedSeq[ActorRef]) { protected def delayedStop(scheduler: Scheduler, abandon: IndexedSeq[ActorRef]): Unit = {
if (abandon.nonEmpty) { if (abandon.nonEmpty) {
if (stopDelay <= Duration.Zero) { if (stopDelay <= Duration.Zero) {
abandon foreach (_ ! PoisonPill) abandon foreach (_ ! PoisonPill)
@ -1327,9 +1274,7 @@ case class DefaultResizer(
* @param capacity current number of routees * @param capacity current number of routees
* @return proposed change in the capacity * @return proposed change in the capacity
*/ */
def filter(pressure: Int, capacity: Int): Int = { def filter(pressure: Int, capacity: Int): Int = rampup(pressure, capacity) + backoff(pressure, capacity)
rampup(pressure, capacity) + backoff(pressure, capacity)
}
/** /**
* Computes a proposed positive (or zero) capacity delta using * Computes a proposed positive (or zero) capacity delta using

View file

@ -14,8 +14,6 @@ import akka.util.NonFatal
import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.ArrayBuffer
import java.io.NotSerializableException import java.io.NotSerializableException
case class NoSerializerFoundException(m: String) extends AkkaException(m)
object Serialization { object Serialization {
/** /**
@ -120,9 +118,7 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
possibilities(0)._2 possibilities(0)._2
} }
serializerMap.putIfAbsent(clazz, ser) match { serializerMap.putIfAbsent(clazz, ser) match {
case null case null log.debug("Using serializer[{}] for message [{}]", ser.getClass.getName, clazz.getName); ser
log.debug("Using serializer[{}] for message [{}]", ser.getClass.getName, clazz.getName)
ser
case some some case some some
} }
case ser ser 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) * A Map of serializer from alias to implementation (class implementing akka.serialization.Serializer)
* By default always contains the following mapping: "java" -> akka.serialization.JavaSerializer * By default always contains the following mapping: "java" -> akka.serialization.JavaSerializer
*/ */
private val serializers: Map[String, Serializer] = { private val serializers: Map[String, Serializer] =
for ((k: String, v: String) settings.Serializers) for ((k: String, v: String) settings.Serializers) yield k -> serializerOf(v).fold(throw _, identity)
yield k -> serializerOf(v).fold(throw _, identity)
}
/** /**
* bindings is a Seq of tuple representing the mapping from Class to Serializer. * bindings is a Seq of tuple representing the mapping from Class to Serializer.

View file

@ -6,7 +6,6 @@ package akka.serialization
import java.io.{ ObjectOutputStream, ByteArrayOutputStream, ObjectInputStream, ByteArrayInputStream } import java.io.{ ObjectOutputStream, ByteArrayOutputStream, ObjectInputStream, ByteArrayInputStream }
import akka.util.ClassLoaderObjectInputStream import akka.util.ClassLoaderObjectInputStream
import akka.actor.DynamicAccess
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
import scala.util.DynamicVariable import scala.util.DynamicVariable

View file

@ -8,6 +8,12 @@ import java.util.concurrent.locks.ReentrantLock
import java.util.concurrent.{ TimeUnit, BlockingQueue } import java.util.concurrent.{ TimeUnit, BlockingQueue }
import java.util.{ AbstractQueue, Queue, Collection, Iterator } 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]( class BoundedBlockingQueue[E <: AnyRef](
val maxCapacity: Int, private val backing: Queue[E]) extends AbstractQueue[E] with BlockingQueue[E] { 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) 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 notEmpty = lock.newCondition()
private val notFull = lock.newCondition() private val notFull = lock.newCondition()

View file

@ -11,6 +11,7 @@ import scala.collection.mutable.{ Builder, WrappedArray }
import scala.collection.immutable.{ IndexedSeq, VectorBuilder } import scala.collection.immutable.{ IndexedSeq, VectorBuilder }
import scala.collection.generic.CanBuildFrom import scala.collection.generic.CanBuildFrom
//FIXME MORE DOCS
object ByteString { object ByteString {
/** /**
@ -53,15 +54,16 @@ object ByteString {
val empty: ByteString = CompactByteString(Array.empty[Byte]) 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] { implicit val canBuildFrom: CanBuildFrom[TraversableOnce[Byte], Byte, ByteString] =
def apply(from: TraversableOnce[Byte]) = newBuilder new CanBuildFrom[TraversableOnce[Byte], Byte, ByteString] {
def apply() = newBuilder def apply(ignore: TraversableOnce[Byte]): ByteStringBuilder = newBuilder
} def apply(): ByteStringBuilder = newBuilder
}
private[akka] object ByteString1C { 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 { final class ByteString1C private (private val bytes: Array[Byte]) extends CompactByteString {
def apply(idx: Int): Byte = bytes(idx) def apply(idx: Int): Byte = bytes(idx)
override def length = bytes.length override def length: Int = bytes.length
def toArray: Array[Byte] = bytes.clone def toArray: Array[Byte] = bytes.clone
@ -81,13 +83,11 @@ object ByteString {
def compact: ByteString1C = this def compact: ByteString1C = this
def asByteBuffer: ByteBuffer = def asByteBuffer: ByteBuffer = toByteString1.asByteBuffer
toByteString1.asByteBuffer
def decodeString(charset: String): String = new String(bytes, charset) def decodeString(charset: String): String = new String(bytes, charset)
def ++(that: ByteString): ByteString = def ++(that: ByteString): ByteString = if (!that.isEmpty) toByteString1 ++ that else this
if (!that.isEmpty) toByteString1 ++ that else this
override def slice(from: Int, until: Int): ByteString = override def slice(from: Int, until: Int): ByteString =
if ((from != 0) || (until != length)) toByteString1.slice(from, until) 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 = override def copyToArray[A >: Byte](xs: Array[A], start: Int, len: Int): Unit =
toByteString1.copyToArray(xs, start, len) toByteString1.copyToArray(xs, start, len)
def copyToBuffer(buffer: ByteBuffer): Int = def copyToBuffer(buffer: ByteBuffer): Int = toByteString1.copyToBuffer(buffer)
toByteString1.copyToBuffer(buffer)
} }
private[akka] object ByteString1 { 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)) def apply(idx: Int): Byte = bytes(checkRangeConvert(idx))
private def checkRangeConvert(index: Int) = { private def checkRangeConvert(index: Int): Int = {
if (0 <= index && length > index) if (0 <= index && length > index)
index + startIndex index + startIndex
else else
@ -128,8 +127,7 @@ object ByteString {
override def clone: CompactByteString = ByteString1C(toArray) override def clone: CompactByteString = ByteString1C(toArray)
def compact: CompactByteString = def compact: CompactByteString = if (length == bytes.length) ByteString1C(bytes) else clone
if (length == bytes.length) ByteString1C(bytes) else clone
def asByteBuffer: ByteBuffer = { def asByteBuffer: ByteBuffer = {
val buffer = ByteBuffer.wrap(bytes, startIndex, length).asReadOnlyBuffer val buffer = ByteBuffer.wrap(bytes, startIndex, length).asReadOnlyBuffer
@ -161,7 +159,6 @@ object ByteString {
if (copyLength > 0) buffer.put(bytes, startIndex, copyLength) if (copyLength > 0) buffer.put(bytes, startIndex, copyLength)
copyLength copyLength
} }
} }
private[akka] object ByteStrings { private[akka] object ByteStrings {
@ -198,10 +195,11 @@ object ByteString {
} }
// 0: both empty, 1: 2nd empty, 2: 1st empty, 3: neither empty // 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 = def compare(b1: ByteString, b2: ByteString): Int =
if (b1.length == 0) if (b1.isEmpty)
if (b2.length == 0) 0 else 2 if (b2.isEmpty) 0 else 2
else if (b2.length == 0) 1 else 3 else if (b2.isEmpty) 1 else 3
} }
@ -439,7 +437,7 @@ final class ByteStringBuilder extends Builder[Byte, ByteString] {
private var _tempLength = 0 private var _tempLength = 0
private var _tempCapacity = 0 private var _tempCapacity = 0
private def clearTemp() { private def clearTemp(): Unit = {
if (_tempLength > 0) { if (_tempLength > 0) {
val arr = new Array[Byte](_tempLength) val arr = new Array[Byte](_tempLength)
Array.copy(_temp, 0, arr, 0, _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) val newtemp = new Array[Byte](size)
if (_tempLength > 0) Array.copy(_temp, 0, newtemp, 0, _tempLength) if (_tempLength > 0) Array.copy(_temp, 0, newtemp, 0, _tempLength)
_temp = newtemp _temp = newtemp
_tempCapacity = _temp.length _tempCapacity = _temp.length
} }
private def ensureTempSize(size: Int) { private def ensureTempSize(size: Int): Unit = {
if (_tempCapacity < size || _tempCapacity == 0) { if (_tempCapacity < size || _tempCapacity == 0) {
var newSize = if (_tempCapacity == 0) 16 else _tempCapacity * 2 var newSize = if (_tempCapacity == 0) 16 else _tempCapacity * 2
while (newSize < size) newSize *= 2 while (newSize < size) newSize *= 2
@ -498,7 +496,7 @@ final class ByteStringBuilder extends Builder[Byte, ByteString] {
this this
} }
def clear() { def clear(): Unit = {
_builder.clear _builder.clear
_length = 0 _length = 0
_tempLength = 0 _tempLength = 0

View file

@ -6,6 +6,13 @@ package akka.util
import java.io.{ InputStream, ObjectInputStream, ObjectStreamClass } 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) { class ClassLoaderObjectInputStream(classLoader: ClassLoader, is: InputStream) extends ObjectInputStream(is) {
override protected def resolveClass(objectStreamClass: ObjectStreamClass): Class[_] = override protected def resolveClass(objectStreamClass: ObjectStreamClass): Class[_] =
try Class.forName(objectStreamClass.getName, false, classLoader) catch { try Class.forName(objectStreamClass.getName, false, classLoader) catch {

View file

@ -3,7 +3,7 @@
*/ */
package akka.util package akka.util
//FIXME DOCS!
object Convert { object Convert {
def intToBytes(value: Int): Array[Byte] = { def intToBytes(value: Int): Array[Byte] = {

View file

@ -5,7 +5,7 @@
package akka.util package akka.util
import java.security.{ MessageDigest, SecureRandom } import java.security.{ MessageDigest, SecureRandom }
//FIXME DOCS
object Crypt { object Crypt {
val hex = "0123456789ABCDEF" val hex = "0123456789ABCDEF"
val lineSeparator = System.getProperty("line.separator") val lineSeparator = System.getProperty("line.separator")
@ -32,7 +32,7 @@ object Crypt {
} }
def hexify(bytes: Array[Byte]): String = { 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)) } bytes.foreach { byte builder.append(hex.charAt((byte & 0xF0) >> 4)).append(hex.charAt(byte & 0xF)) }
builder.toString builder.toString
} }

View file

@ -110,6 +110,7 @@ object Duration {
} }
val Zero: FiniteDuration = new FiniteDuration(0, NANOSECONDS) val Zero: FiniteDuration = new FiniteDuration(0, NANOSECONDS)
val Undefined: Duration = new Duration with Infinite { val Undefined: Duration = new Duration with Infinite {
override def toString = "Duration.Undefined" override def toString = "Duration.Undefined"
override def equals(other: Any) = other.asInstanceOf[AnyRef] eq this override def equals(other: Any) = other.asInstanceOf[AnyRef] eq this
@ -166,8 +167,8 @@ object Duration {
* including itself. * including itself.
*/ */
val Inf: Duration = new Duration with Infinite { val Inf: Duration = new Duration with Infinite {
override def toString = "Duration.Inf" override def toString: String = "Duration.Inf"
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 = MinusInf def unary_- : Duration = MinusInf
} }
@ -177,7 +178,7 @@ object Duration {
*/ */
val MinusInf: Duration = new Duration with Infinite { val MinusInf: Duration = new Duration with Infinite {
override def toString = "Duration.MinusInf" 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 def unary_- : Duration = Inf
} }
@ -188,7 +189,7 @@ object Duration {
def parse(s: String): Duration = unapply(s).get def parse(s: String): Duration = unapply(s).get
implicit object DurationIsOrdered extends Ordering[Duration] { 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 def fromNow: Deadline = Deadline.now + this
// Java API // Java API
def lt(other: Duration) = this < other def lt(other: Duration): Boolean = this < other
def lteq(other: Duration) = this <= other def lteq(other: Duration): Boolean = this <= other
def gt(other: Duration) = this > other def gt(other: Duration): Boolean = this > other
def gteq(other: Duration) = this >= other def gteq(other: Duration): Boolean = this >= other
def plus(other: Duration) = this + other def plus(other: Duration): Duration = this + other
def minus(other: Duration) = this - other def minus(other: Duration): Duration = this - other
def mul(factor: Double) = this * factor def mul(factor: Double): Duration = this * factor
def div(factor: Double) = this / factor def div(factor: Double): Duration = this / factor
def div(other: Duration) = this / other def div(other: Duration): Double = this / other
def neg() = -this def neg(): Duration = -this
def isFinite() = finite_? def isFinite(): Boolean = finite_?
} }
object FiniteDuration { object FiniteDuration {
@ -349,31 +350,19 @@ class FiniteDuration(val length: Long, val unit: TimeUnit) extends Duration {
else c else c
} }
def +(other: Duration) = { def +(other: Duration): Duration = if (!other.finite_?) other else fromNanos(add(toNanos, other.toNanos))
if (!other.finite_?) {
other
} else {
fromNanos(add(toNanos, other.toNanos))
}
}
def -(other: Duration) = { def -(other: Duration): Duration = if (!other.finite_?) other else fromNanos(add(toNanos, -other.toNanos))
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) = override def equals(other: Any) =
(other.asInstanceOf[AnyRef] eq this) || other.isInstanceOf[FiniteDuration] && (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 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 microseconds: FiniteDuration = from(MICROSECONDS)
def nanos = Duration(n, NANOSECONDS) def micros: FiniteDuration = from(MICROSECONDS)
def nanosecond = Duration(n, NANOSECONDS) def microsecond: FiniteDuration = from(MICROSECONDS)
def nano = Duration(n, NANOSECONDS) def micro: FiniteDuration = from(MICROSECONDS)
def microseconds = Duration(n, MICROSECONDS) def milliseconds: FiniteDuration = from(MILLISECONDS)
def micros = Duration(n, MICROSECONDS) def millis: FiniteDuration = from(MILLISECONDS)
def microsecond = Duration(n, MICROSECONDS) def millisecond: FiniteDuration = from(MILLISECONDS)
def micro = Duration(n, MICROSECONDS) def milli: FiniteDuration = from(MILLISECONDS)
def milliseconds = Duration(n, MILLISECONDS) def seconds: FiniteDuration = from(SECONDS)
def millis = Duration(n, MILLISECONDS) def second: FiniteDuration = from(SECONDS)
def millisecond = Duration(n, MILLISECONDS)
def milli = Duration(n, MILLISECONDS)
def seconds = Duration(n, SECONDS) def minutes: FiniteDuration = from(MINUTES)
def second = Duration(n, SECONDS) def minute: FiniteDuration = from(MINUTES)
def minutes = Duration(n, MINUTES) def hours: FiniteDuration = from(HOURS)
def minute = Duration(n, MINUTES) def hour: FiniteDuration = from(HOURS)
def hours = Duration(n, HOURS) def days: FiniteDuration = from(DAYS)
def hour = Duration(n, HOURS) def day: FiniteDuration = from(DAYS)
def days = Duration(n, DAYS) def nanoseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(NANOSECONDS))
def day = Duration(n, DAYS) 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 microseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MICROSECONDS))
def nanos[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS)) def micros[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MICROSECONDS))
def nanosecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS)) def microsecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MICROSECONDS))
def nano[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, NANOSECONDS)) 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 milliseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MILLISECONDS))
def micros[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS)) def millis[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MILLISECONDS))
def microsecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS)) def millisecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MILLISECONDS))
def micro[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MICROSECONDS)) 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 seconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(SECONDS))
def millis[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MILLISECONDS)) def second[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(SECONDS))
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 minutes[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MINUTES))
def second[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, SECONDS)) 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 hours[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(HOURS))
def minute[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, MINUTES)) 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 days[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(DAYS))
def hour[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(Duration(n, HOURS)) def day[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(DAYS))
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 DurationLong(n: Long) { class DurationInt(n: Int) extends DurationOps {
import duration.Classifier override protected def from(timeUnit: TimeUnit): FiniteDuration = Duration(n, timeUnit)
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 DurationDouble(d: Double) { class DurationLong(n: Long) extends DurationOps {
import duration.Classifier override protected def from(timeUnit: TimeUnit): FiniteDuration = Duration(n, timeUnit)
}
def nanoseconds = Duration(d, NANOSECONDS) class DurationDouble(d: Double) extends DurationOps {
def nanos = Duration(d, NANOSECONDS) override protected def from(timeUnit: TimeUnit): FiniteDuration = Duration(d, timeUnit)
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))
} }
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed //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)) 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 { object Timeout {
/** /**
* A timeout with zero duration, will cause most requests to always 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 * 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 * as it may cause memory leaks, blocked threads, or may not even be supported by
* the receiver, which would result in an exception. * 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(timeout: Long): Timeout = new Timeout(timeout)
def apply(length: Long, unit: TimeUnit) = new Timeout(length, unit) def apply(length: Long, unit: TimeUnit): Timeout = new Timeout(length, unit)
implicit def durationToTimeout(duration: Duration) = new Timeout(duration) implicit def durationToTimeout(duration: Duration): Timeout = new Timeout(duration)
implicit def intToTimeout(timeout: Int) = new Timeout(timeout) implicit def intToTimeout(timeout: Int): Timeout = new Timeout(timeout)
implicit def longToTimeout(timeout: Long) = new Timeout(timeout) implicit def longToTimeout(timeout: Long): Timeout = new Timeout(timeout)
} }

View file

@ -45,18 +45,13 @@ object Helpers {
else base64(next, sb) else base64(next, sb)
} }
def ignore[E: Manifest](body: Unit) { //FIXME docs
try { def ignore[E: Manifest](body: Unit): Unit =
body try body catch { case e if manifest[E].erasure.isAssignableFrom(e.getClass) () }
} catch {
case e if manifest[E].erasure.isAssignableFrom(e.getClass) ()
}
}
def withPrintStackTraceOnError(body: Unit) { //FIXME docs
try { def withPrintStackTraceOnError(body: Unit): Unit = {
body try body catch {
} catch {
case e: Throwable case e: Throwable
val sw = new java.io.StringWriter() val sw = new java.io.StringWriter()
var root = e var root = e

View file

@ -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 * 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._ import scala.collection.JavaConversions._
container.entrySet foreach { e e.getValue.foreach(fun(e.getKey, _)) } 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. * 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 * Disassociates the value of type V from the key of type K

View file

@ -24,9 +24,7 @@ class Switch(startAsOn: Boolean = false) {
protected def transcend(from: Boolean, action: Unit): Boolean = synchronized { protected def transcend(from: Boolean, action: Unit): Boolean = synchronized {
if (switch.compareAndSet(from, !from)) { if (switch.compareAndSet(from, !from)) {
try { try action catch {
action
} catch {
case e case e
switch.compareAndSet(!from, from) // revert status switch.compareAndSet(!from, from) // revert status
throw e 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) * 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] = { def ifOnYield[T](action: T): Option[T] = if (switch.get) Some(action) else None
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) * 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] = { def ifOffYield[T](action: T): Option[T] = if (!switch.get) Some(action) else None
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) * 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 switchs lock, i.e. protected from concurrent modification of the switch status. * Executes the given code while holding this switchs 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) * 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) * Returns whether the switch is IMMEDDIATELY off (no locking)
*/ */
def isOff = !isOn def isOff: Boolean = !isOn
} }

View file

@ -6,8 +6,10 @@ package akka.util
/** /**
* Collection of internal reflection utilities which may or may not be * Collection of internal reflection utilities which may or may not be
* available (most services specific to HotSpot, but fails gracefully). * 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 * This optionally holds a function which looks N levels above itself

View file

@ -7,6 +7,9 @@ package akka.util;
import java.lang.reflect.Field; import java.lang.reflect.Field;
/**
* INTERNAL API
*/
public final class Unsafe { public final class Unsafe {
public final static sun.misc.Unsafe instance; public final static sun.misc.Unsafe instance;
static { static {

View file

@ -7,6 +7,7 @@ package akka.util
import scala.util.continuations._ import scala.util.continuations._
import akka.dispatch.MessageDispatcher import akka.dispatch.MessageDispatcher
//FIXME Needs docs
package object cps { package object cps {
def matchC[A, B, C, D](in: A)(pf: PartialFunction[A, B @cpsParam[C, D]]): B @cpsParam[C, D] = pf(in) def matchC[A, B, C, D](in: A)(pf: PartialFunction[A, B @cpsParam[C, D]]): B @cpsParam[C, D] = pf(in)

View file

@ -5,7 +5,7 @@
package akka.util package akka.util
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
//FIXME Needs docs
package object duration { package object duration {
trait Classifier[C] { trait Classifier[C] {
type R type R
@ -15,38 +15,32 @@ package object duration {
object span object span
implicit object spanConvert extends Classifier[span.type] { implicit object spanConvert extends Classifier[span.type] {
type R = FiniteDuration type R = FiniteDuration
def convert(d: FiniteDuration) = d def convert(d: FiniteDuration): FiniteDuration = d
} }
object fromNow object fromNow
implicit object fromNowConvert extends Classifier[fromNow.type] { implicit object fromNowConvert extends Classifier[fromNow.type] {
type R = Deadline 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 intToDurationInt(n: Int): DurationInt = new DurationInt(n)
implicit def longToDurationLong(n: Long) = new DurationLong(n) implicit def longToDurationLong(n: Long): DurationLong = new DurationLong(n)
implicit def doubleToDurationDouble(d: Double) = new DurationDouble(d) implicit def doubleToDurationDouble(d: Double): DurationDouble = new DurationDouble(d)
implicit def pairIntToDuration(p: (Int, TimeUnit)) = Duration(p._1, p._2) implicit def pairIntToDuration(p: (Int, TimeUnit)): FiniteDuration = Duration(p._1, p._2)
implicit def pairLongToDuration(p: (Long, TimeUnit)) = Duration(p._1, p._2) implicit def pairLongToDuration(p: (Long, TimeUnit)): FiniteDuration = Duration(p._1, p._2)
implicit def durationToPair(d: Duration) = (d.length, d.unit) implicit def durationToPair(d: Duration): (Long, TimeUnit) = (d.length, d.unit)
/* /*
* avoid reflection based invocation by using non-duck type * avoid reflection based invocation by using non-duck type
*/ */
class IntMult(i: Int) { class IntMult(i: Int) { def *(d: Duration): Duration = d * i }
def *(d: Duration) = d * i implicit def intMult(i: Int): IntMult = new IntMult(i)
}
implicit def intMult(i: Int) = new IntMult(i)
class LongMult(l: Long) { class LongMult(l: Long) { def *(d: Duration): Duration = d * l }
def *(d: Duration) = d * l implicit def longMult(l: Long): LongMult = new LongMult(l)
}
implicit def longMult(l: Long) = new LongMult(l)
class DoubleMult(f: Double) { class DoubleMult(f: Double) { def *(d: Duration): Duration = d * f }
def *(d: Duration) = d * f implicit def doubleMult(f: Double): DoubleMult = new DoubleMult(f)
}
implicit def doubleMult(f: Double) = new DoubleMult(f)
} }