Merging in master

This commit is contained in:
Viktor Klang 2012-07-04 15:24:23 +02:00
commit 3911b18069
25 changed files with 285 additions and 239 deletions

View file

@ -803,33 +803,6 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll with Defa
assert(Await.result(z, timeout.duration) === 42) assert(Await.result(z, timeout.duration) === 42)
} }
"futureFlowLoops" in {
import Future.flow
import akka.util.cps._
val count = 1000
val promises = List.fill(count)(Promise[Int]())
flow {
var i = 0
val iter = promises.iterator
whileC(iter.hasNext) {
iter.next << i
i += 1
}
}
var i = 0
promises foreach { p
assert(Await.result(p, timeout.duration) === i)
i += 1
}
assert(i === count)
}
"run callbacks async" in { "run callbacks async" in {
val latch = Vector.fill(10)(new TestLatch) val latch = Vector.fill(10)(new TestLatch)

View file

@ -257,7 +257,7 @@ private[akka] object ActorCell {
def cancel() {} def cancel() {}
} }
final val emptyReceiveTimeoutData: (Long, Cancellable) = (-1, emptyCancellable) final val emptyReceiveTimeoutData: (Duration, Cancellable) = (Duration.Undefined, emptyCancellable)
final val emptyBehaviorStack: List[Actor.Receive] = Nil final val emptyBehaviorStack: List[Actor.Receive] = Nil
@ -448,29 +448,24 @@ private[akka] class ActorCell(
final def provider = system.provider final def provider = system.provider
override final def receiveTimeout: Option[Duration] = if (receiveTimeoutData._1 > 0) Some(Duration(receiveTimeoutData._1, MILLISECONDS)) else None override final def receiveTimeout: Option[Duration] = receiveTimeoutData._1 match {
case Duration.Undefined None
override final def setReceiveTimeout(timeout: Duration): Unit = setReceiveTimeout(Some(timeout)) case duration Some(duration)
final def setReceiveTimeout(timeout: Option[Duration]): Unit = {
val timeoutMs = timeout match {
case None -1L
case Some(duration)
val ms = duration.toMillis
if (ms <= 0) -1L
// 1 millisecond is minimum supported
else if (ms < 1) 1L
else ms
}
receiveTimeoutData = (timeoutMs, receiveTimeoutData._2)
} }
final def setReceiveTimeout(timeout: Option[Duration]): Unit = setReceiveTimeout(timeout.getOrElse(Duration.Undefined))
override final def setReceiveTimeout(timeout: Duration): Unit =
receiveTimeoutData = (
if (Duration.Undefined == timeout || timeout.toMillis < 1) Duration.Undefined else timeout,
receiveTimeoutData._2)
final override def resetReceiveTimeout(): Unit = setReceiveTimeout(None) final override def resetReceiveTimeout(): Unit = setReceiveTimeout(None)
/** /**
* In milliseconds * In milliseconds
*/ */
var receiveTimeoutData: (Long, Cancellable) = emptyReceiveTimeoutData var receiveTimeoutData: (Duration, Cancellable) = emptyReceiveTimeoutData
@volatile @volatile
private var _childrenRefsDoNotCallMeDirectly: ChildrenContainer = EmptyChildrenContainer private var _childrenRefsDoNotCallMeDirectly: ChildrenContainer = EmptyChildrenContainer
@ -1014,10 +1009,10 @@ private[akka] class ActorCell(
final def checkReceiveTimeout() { final def checkReceiveTimeout() {
val recvtimeout = receiveTimeoutData val recvtimeout = receiveTimeoutData
if (recvtimeout._1 > 0 && !mailbox.hasMessages) { if (Duration.Undefined != recvtimeout._1 && !mailbox.hasMessages) {
recvtimeout._2.cancel() //Cancel any ongoing future recvtimeout._2.cancel() //Cancel any ongoing future
//Only reschedule if desired and there are currently no more messages to be processed //Only reschedule if desired and there are currently no more messages to be processed
receiveTimeoutData = (recvtimeout._1, system.scheduler.scheduleOnce(Duration(recvtimeout._1, TimeUnit.MILLISECONDS), self, ReceiveTimeout)) receiveTimeoutData = (recvtimeout._1, system.scheduler.scheduleOnce(recvtimeout._1, self, ReceiveTimeout))
} else cancelReceiveTimeout() } else cancelReceiveTimeout()
} }

View file

@ -1,85 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
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)
def loopC[A, U](block: U @cps[A])(implicit loop: CPSLoop[A], dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[A] =
loop.loopC(block)
def whileC[A, U](test: Boolean)(block: U @cps[A])(implicit loop: CPSLoop[A], dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[A] =
loop.whileC(test)(block)
def repeatC[A, U](times: Int)(block: U @cps[A])(implicit loop: CPSLoop[A], dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[A] =
loop.repeatC(times)(block)
}
package cps {
object CPSLoop extends DefaultCPSLoop {
implicit object FutureCPSLoop extends FutureCPSLoop
}
trait CPSLoop[A] {
def loopC[U](block: U @cps[A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[A]
def whileC[U](test: Boolean)(block: U @cps[A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[A]
def repeatC[U](times: Int)(block: U @cps[A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[A]
}
import akka.dispatch.{ Future, Promise }
class FutureCPSLoop extends CPSLoop[Future[Any]] {
def loopC[U](block: U @cps[Future[Any]])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[Future[Any]] =
shift { c: (Unit Future[Any])
Future(reify(block) flatMap (_ reify(loopC(block))) foreach c)
}
def whileC[U](test: Boolean)(block: U @cps[Future[Any]])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[Future[Any]] =
shift { c: (Unit Future[Any])
if (test)
Future(reify(block) flatMap (_ reify(whileC(test)(block))) foreach c)
else
Promise() success (shiftUnitR[Unit, Future[Any]](()) foreach c)
}
def repeatC[U](times: Int)(block: U @cps[Future[Any]])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[Future[Any]] =
shift { c: (Unit Future[Any])
if (times > 0)
Future(reify(block) flatMap (_ reify(repeatC(times - 1)(block))) foreach c)
else
Promise() success (shiftUnitR[Unit, Future[Any]](()) foreach c)
}
}
trait DefaultCPSLoop {
implicit def defaultCPSLoop[A] = new CPSLoop[A] {
def loopC[U](block: U @cps[A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[A] = {
block
loopC(block)
}
def whileC[U](test: Boolean)(block: U @cps[A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[A] = {
if (test) {
block
whileC(test)(block)
}
}
def repeatC[U](times: Int)(block: U @cps[A])(implicit dispatcher: MessageDispatcher, timeout: Timeout): Unit @cps[A] = {
if (times > 0) {
block
repeatC(times - 1)(block)
}
}
}
}
}

View file

@ -2,13 +2,13 @@ package akka.agent
import language.postfixOps import language.postfixOps
import scala.concurrent.Await import scala.concurrent.{ Await, Future }
import scala.concurrent.util.Duration import scala.concurrent.util.Duration
import scala.concurrent.util.duration._ import scala.concurrent.util.duration._
import akka.util.Timeout import akka.util.Timeout
import akka.testkit._ import akka.testkit._
import scala.concurrent.stm._ import scala.concurrent.stm._
import java.util.concurrent.CountDownLatch import java.util.concurrent.{ CountDownLatch, TimeUnit }
class CountDownFunction[A](num: Int = 1) extends Function1[A, A] { class CountDownFunction[A](num: Int = 1) extends Function1[A, A] {
val latch = new CountDownLatch(num) val latch = new CountDownLatch(num)
@ -38,14 +38,15 @@ class AgentSpec extends AkkaSpec {
"maintain order between send and sendOff" in { "maintain order between send and sendOff" in {
val countDown = new CountDownFunction[String] val countDown = new CountDownFunction[String]
val l1, l2 = new CountDownLatch(1)
val agent = Agent("a") val agent = Agent("a")
agent send (_ + "b") agent send (_ + "b")
val longRunning = (s: String) { Thread.sleep(2000); s + "c" } agent.sendOff((s: String) { l1.countDown; l2.await(5, TimeUnit.SECONDS); s + "c" })
agent sendOff longRunning l1.await(5, TimeUnit.SECONDS)
agent send (_ + "d") agent send (_ + "d")
agent send countDown agent send countDown
l2.countDown
countDown.await(5 seconds) countDown.await(5 seconds)
agent() must be("abcd") agent() must be("abcd")
@ -53,16 +54,17 @@ class AgentSpec extends AkkaSpec {
} }
"maintain order between alter and alterOff" in { "maintain order between alter and alterOff" in {
val l1, l2 = new CountDownLatch(1)
val agent = Agent("a") val agent = Agent("a")
val r1 = agent.alter(_ + "b")(5000) val r1 = agent.alter(_ + "b")(5000)
val r2 = agent.alterOff((s: String) { Thread.sleep(2000); s + "c" })(5000) val r2 = agent.alterOff((s: String) { l1.countDown; l2.await(5, TimeUnit.SECONDS); s + "c" })(5000)
l1.await(5, TimeUnit.SECONDS)
val r3 = agent.alter(_ + "d")(5000) val r3 = agent.alter(_ + "d")(5000)
val result = Future.sequence(Seq(r1, r2, r3)).map(_.mkString(":"))
l2.countDown
Await.result(r1, 5 seconds) must be === "ab" Await.result(result, 5 seconds) must be === "ab:abc:abcd"
Await.result(r2, 5 seconds) must be === "abc"
Await.result(r3, 5 seconds) must be === "abcd"
agent() must be("abcd") agent() must be("abcd")

View file

@ -21,10 +21,10 @@ class ProducerRegistryTest extends WordSpec with MustMatchers with SharedCamelSy
"register a started SendProcessor for the producer, which is stopped when the actor is stopped" in { "register a started SendProcessor for the producer, which is stopped when the actor is stopped" in {
val actorRef = newEmptyActor val actorRef = newEmptyActor
val processor = registerProcessorFor(actorRef) val processor = registerProcessorFor(actorRef)
camel.awaitActivation(actorRef, 1 second) camel.awaitActivation(actorRef, 5 second)
processor.isStarted must be(true) processor.isStarted must be(true)
system.stop(actorRef) system.stop(actorRef)
camel.awaitDeactivation(actorRef, 1 second) camel.awaitDeactivation(actorRef, 5 second)
(processor.isStopping || processor.isStopped) must be(true) (processor.isStopping || processor.isStopped) must be(true)
} }
"remove and stop the SendProcessor if the actorRef is registered" in { "remove and stop the SendProcessor if the actorRef is registered" in {

View file

@ -25,7 +25,7 @@ akka {
# Should the 'leader' in the cluster be allowed to automatically mark unreachable nodes as DOWN? # Should the 'leader' in the cluster be allowed to automatically mark unreachable nodes as DOWN?
# Using auto-down implies that two separate clusters will automatically be formed in case of # Using auto-down implies that two separate clusters will automatically be formed in case of
# network partition. # network partition.
auto-down = on auto-down = off
# the number of gossip daemon actors # the number of gossip daemon actors
nr-of-gossip-daemons = 4 nr-of-gossip-daemons = 4

View file

@ -18,10 +18,8 @@ object LeaderLeavingMultiJvmSpec extends MultiNodeConfig {
commonConfig( commonConfig(
debugConfig(on = false) debugConfig(on = false)
.withFallback(ConfigFactory.parseString(""" .withFallback(ConfigFactory.parseString("""
akka.cluster { # turn off unreachable reaper
leader-actions-interval = 5 s # increase the leader action task frequency to make sure we get a chance to test the LEAVING state akka.cluster.unreachable-nodes-reaper-interval = 300 s""")
unreachable-nodes-reaper-interval = 30 s
}""")
.withFallback(MultiNodeClusterSpec.clusterConfig))) .withFallback(MultiNodeClusterSpec.clusterConfig)))
} }
@ -35,7 +33,7 @@ abstract class LeaderLeavingSpec
import LeaderLeavingMultiJvmSpec._ import LeaderLeavingMultiJvmSpec._
val leaderHandoffWaitingTime = 30.seconds.dilated val leaderHandoffWaitingTime = 30.seconds
"A LEADER that is LEAVING" must { "A LEADER that is LEAVING" must {
@ -45,41 +43,60 @@ abstract class LeaderLeavingSpec
val oldLeaderAddress = cluster.leader val oldLeaderAddress = cluster.leader
if (cluster.isLeader) { within(leaderHandoffWaitingTime) {
cluster.leave(oldLeaderAddress) if (cluster.isLeader) {
enterBarrier("leader-left")
// verify that a NEW LEADER have taken over enterBarrier("registered-listener")
awaitCond(!cluster.isLeader)
// verify that the LEADER is shut down cluster.leave(oldLeaderAddress)
awaitCond(!cluster.isRunning, 30.seconds.dilated) enterBarrier("leader-left")
// verify that the LEADER is REMOVED // verify that a NEW LEADER have taken over
awaitCond(cluster.status == MemberStatus.Removed) awaitCond(!cluster.isLeader)
} else { // verify that the LEADER is shut down
awaitCond(!cluster.isRunning)
enterBarrier("leader-left") // verify that the LEADER is REMOVED
awaitCond(cluster.status == MemberStatus.Removed)
// verify that the LEADER is LEAVING } else {
awaitCond(cluster.latestGossip.members.exists(m m.status == MemberStatus.Leaving && m.address == oldLeaderAddress), leaderHandoffWaitingTime) // wait on LEAVING
// verify that the LEADER is EXITING val leavingLatch = TestLatch()
awaitCond(cluster.latestGossip.members.exists(m m.status == MemberStatus.Exiting && m.address == oldLeaderAddress), leaderHandoffWaitingTime) // wait on EXITING val exitingLatch = TestLatch()
val expectedAddresses = roles.toSet map address
cluster.registerListener(new MembershipChangeListener {
def notify(members: SortedSet[Member]) {
def check(status: MemberStatus): Boolean =
(members.map(_.address) == expectedAddresses &&
members.exists(m m.address == oldLeaderAddress && m.status == status))
if (check(MemberStatus.Leaving)) leavingLatch.countDown()
if (check(MemberStatus.Exiting)) exitingLatch.countDown()
}
})
enterBarrier("registered-listener")
// verify that the LEADER is no longer part of the 'members' set enterBarrier("leader-left")
awaitCond(cluster.latestGossip.members.forall(_.address != oldLeaderAddress), leaderHandoffWaitingTime)
// verify that the LEADER is not part of the 'unreachable' set // verify that the LEADER is LEAVING
awaitCond(cluster.latestGossip.overview.unreachable.forall(_.address != oldLeaderAddress), leaderHandoffWaitingTime) leavingLatch.await
// verify that we have a new LEADER // verify that the LEADER is EXITING
awaitCond(cluster.leader != oldLeaderAddress, leaderHandoffWaitingTime) exitingLatch.await
// verify that the LEADER is no longer part of the 'members' set
awaitCond(cluster.latestGossip.members.forall(_.address != oldLeaderAddress))
// verify that the LEADER is not part of the 'unreachable' set
awaitCond(cluster.latestGossip.overview.unreachable.forall(_.address != oldLeaderAddress))
// verify that we have a new LEADER
awaitCond(cluster.leader != oldLeaderAddress)
}
enterBarrier("finished")
} }
enterBarrier("finished")
} }
} }
} }

View file

@ -18,11 +18,8 @@ object NodeLeavingAndExitingMultiJvmSpec extends MultiNodeConfig {
commonConfig( commonConfig(
debugConfig(on = false) debugConfig(on = false)
.withFallback(ConfigFactory.parseString(""" .withFallback(ConfigFactory.parseString("""
akka.cluster { # turn off unreachable reaper
leader-actions-interval = 5 s # increase the leader action task frequency to make sure we get a chance to test the LEAVING state akka.cluster.unreachable-nodes-reaper-interval = 300 s""")
unreachable-nodes-reaper-interval = 300 s # turn "off"
}
""")
.withFallback(MultiNodeClusterSpec.clusterConfig))) .withFallback(MultiNodeClusterSpec.clusterConfig)))
} }
@ -42,26 +39,39 @@ abstract class NodeLeavingAndExitingSpec
awaitClusterUp(first, second, third) awaitClusterUp(first, second, third)
runOn(first) {
cluster.leave(second)
}
enterBarrier("second-left")
runOn(first, third) { runOn(first, third) {
val secondAddess = address(second)
val leavingLatch = TestLatch()
val exitingLatch = TestLatch()
val expectedAddresses = roles.toSet map address
cluster.registerListener(new MembershipChangeListener {
def notify(members: SortedSet[Member]) {
def check(status: MemberStatus): Boolean =
(members.map(_.address) == expectedAddresses &&
members.exists(m m.address == secondAddess && m.status == status))
if (check(MemberStatus.Leaving)) leavingLatch.countDown()
if (check(MemberStatus.Exiting)) exitingLatch.countDown()
}
})
enterBarrier("registered-listener")
// 1. Verify that 'second' node is set to LEAVING runOn(third) {
// We have set the 'leader-actions-interval' to 5 seconds to make sure that we get a cluster.leave(second)
// chance to test the LEAVING state before the leader moves the node to EXITING }
awaitCond(cluster.latestGossip.members.exists(_.status == MemberStatus.Leaving)) // wait on LEAVING enterBarrier("second-left")
val hasLeft = cluster.latestGossip.members.find(_.status == MemberStatus.Leaving) // verify node that left
hasLeft must be('defined)
hasLeft.get.address must be(address(second))
// 2. Verify that 'second' node is set to EXITING // Verify that 'second' node is set to LEAVING
awaitCond(cluster.latestGossip.members.exists(_.status == MemberStatus.Exiting)) // wait on EXITING leavingLatch.await
val hasExited = cluster.latestGossip.members.find(_.status == MemberStatus.Exiting) // verify node that exited
hasExited must be('defined) // Verify that 'second' node is set to EXITING
hasExited.get.address must be(address(second)) exitingLatch.await
}
// node that is leaving
runOn(second) {
enterBarrier("registered-listener")
enterBarrier("second-left")
} }
enterBarrier("finished") enterBarrier("finished")

View file

@ -33,7 +33,7 @@ class ClusterConfigSpec extends AkkaSpec {
JoinTimeout must be(60 seconds) JoinTimeout must be(60 seconds)
NrOfGossipDaemons must be(4) NrOfGossipDaemons must be(4)
AutoJoin must be(true) AutoJoin must be(true)
AutoDown must be(true) AutoDown must be(false)
GossipDifferentViewProbability must be(0.8 plusOrMinus 0.0001) GossipDifferentViewProbability must be(0.8 plusOrMinus 0.0001)
SchedulerTickDuration must be(33 millis) SchedulerTickDuration must be(33 millis)
SchedulerTicksPerWheel must be(512) SchedulerTicksPerWheel must be(512)

View file

@ -235,7 +235,7 @@ public class UntypedActorDocTestBase {
final ArrayList<Future<Object>> futures = new ArrayList<Future<Object>>(); final ArrayList<Future<Object>> futures = new ArrayList<Future<Object>>();
futures.add(ask(actorA, "request", 1000)); // using 1000ms timeout futures.add(ask(actorA, "request", 1000)); // using 1000ms timeout
futures.add(ask(actorB, "reqeest", t)); // using timeout from above futures.add(ask(actorB, "another request", t)); // using timeout from above
final Future<Iterable<Object>> aggregate = Futures.sequence(futures, system.dispatcher()); final Future<Iterable<Object>> aggregate = Futures.sequence(futures, system.dispatcher());

View file

@ -12,11 +12,19 @@ service applications that integrates with Akka. It provides a REST API on top of
Getting started Getting started
--------------- ---------------
First you must make your application aware of play-mini. Easiest way to get started with `Play2 Mini <https://github.com/typesafehub/play2-mini>`_ is to use the
In SBT you just have to add the following to your ``libraryDependencies``:: G8 project templates, as described in the `Play2 Mini Documentation <https://github.com/typesafehub/play2-mini>`_.
If you already have an Akka project and want to add Play2 Mini, you must first add the following to
your ``libraryDependencies``::
libraryDependencies += "com.typesafe" %% "play-mini" % "<version-number>" libraryDependencies += "com.typesafe" %% "play-mini" % "<version-number>"
In case you need to start Play2 Mini programatically you can use::
play.core.server.NettyServer.main(Array())
Akka Mist Akka Mist
========= =========

View file

@ -66,7 +66,6 @@ class ConfigNamespaceHandlerTest extends WordSpec with MustMatchers with PojoSRT
"set up ActorSystem when bundle starts" in { "set up ActorSystem when bundle starts" in {
val system = serviceForType[ActorSystem] val system = serviceForType[ActorSystem]
system must not be (null) system must not be (null)
system.settings.config.getString("some.config.key") must be("value") system.settings.config.getString("some.config.key") must be("value")
} }

View file

@ -14,8 +14,8 @@ import java.net.URL
import java.util.jar.JarInputStream import java.util.jar.JarInputStream
import java.io.{ FileInputStream, FileOutputStream, File } import java.io.{ FileInputStream, FileOutputStream, File }
import java.util.{ Date, ServiceLoader, HashMap }
import org.scalatest.{ BeforeAndAfterAll, Suite } import org.scalatest.{ BeforeAndAfterAll, Suite }
import java.util.{ UUID, Date, ServiceLoader, HashMap }
/** /**
* Trait that provides support for building akka-osgi tests using PojoSR * Trait that provides support for building akka-osgi tests using PojoSR
@ -29,11 +29,11 @@ trait PojoSRTestSupport extends Suite with BeforeAndAfterAll {
* All bundles being found on the test classpath are automatically installed and started in the PojoSR runtime. * All bundles being found on the test classpath are automatically installed and started in the PojoSR runtime.
* Implement this to define the extra bundles that should be available for testing. * Implement this to define the extra bundles that should be available for testing.
*/ */
val testBundles: Seq[BundleDescriptor] def testBundles: Seq[BundleDescriptor]
lazy val context: BundleContext = { lazy val context: BundleContext = {
val config = new HashMap[String, AnyRef]() val config = new HashMap[String, AnyRef]()
System.setProperty("org.osgi.framework.storage", "target/akka-osgi/" + System.currentTimeMillis) System.setProperty("org.osgi.framework.storage", "target/akka-osgi/" + UUID.randomUUID().toString)
val bundles = new ClasspathScanner().scanForBundles() val bundles = new ClasspathScanner().scanForBundles()
bundles.addAll(testBundles) bundles.addAll(testBundles)

View file

@ -209,6 +209,7 @@ akka {
# "" or SecureRandom => (default) # "" or SecureRandom => (default)
# "SHA1PRNG" => Can be slow because of blocking issues on Linux # "SHA1PRNG" => Can be slow because of blocking issues on Linux
# "AES128CounterSecureRNG" => fastest startup and based on AES encryption algorithm # "AES128CounterSecureRNG" => fastest startup and based on AES encryption algorithm
# "AES256CounterSecureRNG"
# The following use one of 3 possible seed sources, depending on availability: /dev/random, random.org and SecureRandom (provided by Java) # The following use one of 3 possible seed sources, depending on availability: /dev/random, random.org and SecureRandom (provided by Java)
# "AES128CounterInetRNG" # "AES128CounterInetRNG"
# "AES256CounterInetRNG" (Install JCE Unlimited Strength Jurisdiction Policy Files first) # "AES256CounterInetRNG" (Install JCE Unlimited Strength Jurisdiction Policy Files first)

View file

@ -38,7 +38,7 @@ private[akka] object NettySSLSupport {
} }
val rng = rngName match { val rng = rngName match {
case Some(r @ ("AES128CounterSecureRNG" | "AES128CounterInetRNG" | "AES256CounterInetRNG")) case Some(r @ ("AES128CounterSecureRNG" | "AES256CounterSecureRNG" | "AES128CounterInetRNG" | "AES256CounterInetRNG"))
log.debug("SSL random number generator set to: {}", r) log.debug("SSL random number generator set to: {}", r)
SecureRandom.getInstance(r, AkkaProvider) SecureRandom.getInstance(r, AkkaProvider)
case Some(s @ ("SHA1PRNG" | "NativePRNG")) case Some(s @ ("SHA1PRNG" | "NativePRNG"))

View file

@ -3,17 +3,18 @@
*/ */
package akka.security.provider package akka.security.provider
import org.uncommons.maths.random.{ AESCounterRNG, DefaultSeedGenerator } import org.uncommons.maths.random.{ AESCounterRNG }
import SeedSize.Seed128
/** /**
* Internal API * Internal API
* This class is a wrapper around the 128-bit AESCounterRNG algorithm provided by http://maths.uncommons.org/ * This class is a wrapper around the 128-bit AESCounterRNG algorithm provided by http://maths.uncommons.org/
* It uses the default seed generator which uses one of the following 3 random seed sources: * It uses the default seed generator which uses one of the following 3 random seed sources:
* Depending on availability: /dev/random, random.org and SecureRandom (provided by Java) * Depending on availability: random.org, /dev/random, and SecureRandom (provided by Java)
* The only method used by netty ssl is engineNextBytes(bytes) * The only method used by netty ssl is engineNextBytes(bytes)
*/ */
class AES128CounterInetRNG extends java.security.SecureRandomSpi { class AES128CounterInetRNG extends java.security.SecureRandomSpi {
private val rng = new AESCounterRNG() private val rng = new AESCounterRNG(engineGenerateSeed(Seed128))
/** /**
* This is managed internally by AESCounterRNG * This is managed internally by AESCounterRNG
@ -35,6 +36,6 @@ class AES128CounterInetRNG extends java.security.SecureRandomSpi {
* @param numBytes the number of seed bytes to generate. * @param numBytes the number of seed bytes to generate.
* @return the seed bytes. * @return the seed bytes.
*/ */
override protected def engineGenerateSeed(numBytes: Int): Array[Byte] = DefaultSeedGenerator.getInstance.generateSeed(numBytes) override protected def engineGenerateSeed(numBytes: Int): Array[Byte] = InternetSeedGenerator.getInstance.generateSeed(numBytes)
} }

View file

@ -4,15 +4,22 @@
package akka.security.provider package akka.security.provider
import org.uncommons.maths.random.{ AESCounterRNG, SecureRandomSeedGenerator } import org.uncommons.maths.random.{ AESCounterRNG, SecureRandomSeedGenerator }
import SeedSize.Seed128
/** /**
* Internal API * Internal API
* This class is a wrapper around the AESCounterRNG algorithm provided by http://maths.uncommons.org/ * * This class is a wrapper around the 128-bit AESCounterRNG algorithm provided by http://maths.uncommons.org/
* The only method used by netty ssl is engineNextBytes(bytes) * The only method used by netty ssl is engineNextBytes(bytes)
* This RNG is good to use to prevent startup delay when you don't have Internet access to random.org * This RNG is good to use to prevent startup delay when you don't have Internet access to random.org
*/ */
class AES128CounterSecureRNG extends java.security.SecureRandomSpi { class AES128CounterSecureRNG extends java.security.SecureRandomSpi {
private val rng = new AESCounterRNG(new SecureRandomSeedGenerator()) /**Singleton instance. */
private final val Instance: SecureRandomSeedGenerator = new SecureRandomSeedGenerator
/**
* Make sure the seed generator is provided by a SecureRandom singleton and not default 'Random'
*/
private val rng = new AESCounterRNG(engineGenerateSeed(Seed128))
/** /**
* This is managed internally by AESCounterRNG * This is managed internally by AESCounterRNG
@ -34,6 +41,6 @@ class AES128CounterSecureRNG extends java.security.SecureRandomSpi {
* @param numBytes the number of seed bytes to generate. * @param numBytes the number of seed bytes to generate.
* @return the seed bytes. * @return the seed bytes.
*/ */
override protected def engineGenerateSeed(numBytes: Int): Array[Byte] = (new SecureRandomSeedGenerator()).generateSeed(numBytes) override protected def engineGenerateSeed(numBytes: Int): Array[Byte] = Instance.generateSeed(numBytes)
} }

View file

@ -3,23 +3,18 @@
*/ */
package akka.security.provider package akka.security.provider
import org.uncommons.maths.random.{ AESCounterRNG, DefaultSeedGenerator } import org.uncommons.maths.random.{ AESCounterRNG }
import SeedSize.Seed256
/** /**
* Internal API * Internal API
* This class is a wrapper around the 256-bit AESCounterRNG algorithm provided by http://maths.uncommons.org/ * This class is a wrapper around the 256-bit AESCounterRNG algorithm provided by http://maths.uncommons.org/
* It uses the default seed generator which uses one of the following 3 random seed sources: * It uses the default seed generator which uses one of the following 3 random seed sources:
* Depending on availability: /dev/random, random.org and SecureRandom (provided by Java) * Depending on availability: random.org, /dev/random, and SecureRandom (provided by Java)
* The only method used by netty ssl is engineNextBytes(bytes) * The only method used by netty ssl is engineNextBytes(bytes)
*/ */
class AES256CounterInetRNG extends java.security.SecureRandomSpi { class AES256CounterInetRNG extends java.security.SecureRandomSpi {
/** private val rng = new AESCounterRNG(engineGenerateSeed(Seed256))
* From AESCounterRNG API docs:
* Valid values are 16 (128 bits), 24 (192 bits) and 32 (256 bits).
* Any other values will result in an exception from the AES implementation.
*/
private val AES_256_BIT = 32 // Magic number is magic
private val rng = new AESCounterRNG(AES_256_BIT)
/** /**
* This is managed internally by AESCounterRNG * This is managed internally by AESCounterRNG
@ -41,6 +36,6 @@ class AES256CounterInetRNG extends java.security.SecureRandomSpi {
* @param numBytes the number of seed bytes to generate. * @param numBytes the number of seed bytes to generate.
* @return the seed bytes. * @return the seed bytes.
*/ */
override protected def engineGenerateSeed(numBytes: Int): Array[Byte] = DefaultSeedGenerator.getInstance.generateSeed(numBytes) override protected def engineGenerateSeed(numBytes: Int): Array[Byte] = InternetSeedGenerator.getInstance.generateSeed(numBytes)
} }

View file

@ -0,0 +1,43 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.security.provider
import org.uncommons.maths.random.{ AESCounterRNG, SecureRandomSeedGenerator }
import SeedSize.Seed256
/**
* Internal API
* This class is a wrapper around the 256-bit AESCounterRNG algorithm provided by http://maths.uncommons.org/
* The only method used by netty ssl is engineNextBytes(bytes)
* This RNG is good to use to prevent startup delay when you don't have Internet access to random.org
*/
class AES256CounterSecureRNG extends java.security.SecureRandomSpi {
/**Singleton instance. */
private final val Instance: SecureRandomSeedGenerator = new SecureRandomSeedGenerator
private val rng = new AESCounterRNG(engineGenerateSeed(Seed256))
/**
* This is managed internally by AESCounterRNG
*/
override protected def engineSetSeed(seed: Array[Byte]): Unit = ()
/**
* Generates a user-specified number of random bytes.
*
* @param bytes the array to be filled in with random bytes.
*/
override protected def engineNextBytes(bytes: Array[Byte]): Unit = rng.nextBytes(bytes)
/**
* Unused method
* Returns the given number of seed bytes. This call may be used to
* seed other random number generators.
*
* @param numBytes the number of seed bytes to generate.
* @return the seed bytes.
*/
override protected def engineGenerateSeed(numBytes: Int): Array[Byte] = Instance.generateSeed(numBytes)
}

View file

@ -13,11 +13,13 @@ object AkkaProvider extends Provider("Akka", 1.0, "Akka provider 1.0 that implem
def run = { def run = {
//SecureRandom //SecureRandom
put("SecureRandom.AES128CounterSecureRNG", classOf[AES128CounterSecureRNG].getName) put("SecureRandom.AES128CounterSecureRNG", classOf[AES128CounterSecureRNG].getName)
put("SecureRandom.AES256CounterSecureRNG", classOf[AES256CounterSecureRNG].getName)
put("SecureRandom.AES128CounterInetRNG", classOf[AES128CounterInetRNG].getName) put("SecureRandom.AES128CounterInetRNG", classOf[AES128CounterInetRNG].getName)
put("SecureRandom.AES256CounterInetRNG", classOf[AES256CounterInetRNG].getName) put("SecureRandom.AES256CounterInetRNG", classOf[AES256CounterInetRNG].getName)
//Implementation type: software or hardware //Implementation type: software or hardware
put("SecureRandom.AES128CounterSecureRNG ImplementedIn", "Software") put("SecureRandom.AES128CounterSecureRNG ImplementedIn", "Software")
put("SecureRandom.AES256CounterSecureRNG ImplementedIn", "Software")
put("SecureRandom.AES128CounterInetRNG ImplementedIn", "Software") put("SecureRandom.AES128CounterInetRNG ImplementedIn", "Software")
put("SecureRandom.AES256CounterInetRNG ImplementedIn", "Software") put("SecureRandom.AES256CounterInetRNG ImplementedIn", "Software")
null //Magic null is magic null //Magic null is magic

View file

@ -0,0 +1,53 @@
// ============================================================================
// Copyright 2006-2010 Daniel W. Dyer
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
// ============================================================================
package akka.security.provider
import org.uncommons.maths.random.{ SeedGenerator, SeedException, SecureRandomSeedGenerator, RandomDotOrgSeedGenerator, DevRandomSeedGenerator }
/**
* Internal API
* Seed generator that maintains multiple strategies for seed
* generation and will delegate to the best one available for the
* current operating environment.
* @author Daniel Dyer
*/
object InternetSeedGenerator {
/**
* @return The singleton instance of this class.
*/
def getInstance: InternetSeedGenerator = Instance
/**Singleton instance. */
private final val Instance: InternetSeedGenerator = new InternetSeedGenerator
/**Delegate generators. */
private final val Generators: Seq[SeedGenerator] =
Seq(new RandomDotOrgSeedGenerator, // first try the Internet seed generator
new SecureRandomSeedGenerator) // this is last because it always works
}
final class InternetSeedGenerator extends SeedGenerator {
/**
* Generates a seed by trying each of the available strategies in
* turn until one succeeds. Tries the most suitable strategy first
* and eventually degrades to the least suitable (but guaranteed to
* work) strategy.
* @param length The length (in bytes) of the seed.
* @return A random seed of the requested length.
*/
def generateSeed(length: Int): Array[Byte] = InternetSeedGenerator.Generators.view.flatMap(
g try Option(g.generateSeed(length)) catch { case _: SeedException None }).headOption.getOrElse(throw new IllegalStateException("All available seed generation strategies failed."))
}

View file

@ -0,0 +1,20 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.security.provider
/**
* Internal API
* From AESCounterRNG API docs:
* Valid values are 16 (128 bits), 24 (192 bits) and 32 (256 bits).
* Any other values will result in an exception from the AES implementation.
*
* Internal API
*/
private[provider] object SeedSize {
val Seed128 = 16
val Seed192 = 24
val Seed256 = 32
}

View file

@ -87,6 +87,9 @@ class Ticket1978SHA1PRNGSpec extends Ticket1978CommunicationSpec(getCipherConfig
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class Ticket1978AES128CounterSecureRNGSpec extends Ticket1978CommunicationSpec(getCipherConfig("AES128CounterSecureRNG", "TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA")) class Ticket1978AES128CounterSecureRNGSpec extends Ticket1978CommunicationSpec(getCipherConfig("AES128CounterSecureRNG", "TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA"))
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class Ticket1978AES256CounterSecureRNGSpec extends Ticket1978CommunicationSpec(getCipherConfig("AES256CounterSecureRNG", "TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA"))
/** /**
* Both of the <quote>Inet</quote> variants require access to the Internet to access random.org. * Both of the <quote>Inet</quote> variants require access to the Internet to access random.org.
*/ */

View file

@ -43,6 +43,20 @@ object AkkaBuild extends Build {
Publish.defaultPublishTo in ThisBuild <<= crossTarget / "repository", Publish.defaultPublishTo in ThisBuild <<= crossTarget / "repository",
Unidoc.unidocExclude := Seq(samples.id, tutorials.id), Unidoc.unidocExclude := Seq(samples.id, tutorials.id),
Dist.distExclude := Seq(actorTests.id, akkaSbtPlugin.id, docs.id), Dist.distExclude := Seq(actorTests.id, akkaSbtPlugin.id, docs.id),
initialCommands in ThisBuild :=
"""|import akka.actor._
|import akka.dispatch._
|import com.typesafe.config.ConfigFactory
|import akka.util.duration._
|import akka.util.Timeout
|val config = ConfigFactory.parseString("akka.stdout-loglevel=INFO,akka.loglevel=DEBUG")
|val remoteConfig = ConfigFactory.parseString("akka.remote.netty{port=0,use-dispatcher-for-io=akka.actor.default-dispatcher,execution-pool-size=0},akka.actor.provider=RemoteActorRefProvider").withFallback(config)
|var system: ActorSystem = null
|def startSystem(remoting: Boolean = false) { system = ActorSystem("repl", if(remoting) remoteConfig else config); println("dont forget to system.shutdown()!") }
|implicit def ec = system.dispatcher
|implicit val timeout = Timeout(5 seconds)
|""".stripMargin,
initialCommands in Test in ThisBuild += "import akka.testkit._",
// online version of docs // online version of docs
sphinxDocs <<= baseDirectory / "akka-docs", sphinxDocs <<= baseDirectory / "akka-docs",
sphinxTags in sphinxHtml += "online", sphinxTags in sphinxHtml += "online",
@ -76,6 +90,7 @@ object AkkaBuild extends Build {
dependencies = Seq(actor), dependencies = Seq(actor),
settings = defaultSettings ++ Seq( settings = defaultSettings ++ Seq(
libraryDependencies ++= Dependencies.testkit, libraryDependencies ++= Dependencies.testkit,
initialCommands += "import akka.testkit._",
previousArtifact := akkaPreviousArtifact("akka-testkit") previousArtifact := akkaPreviousArtifact("akka-testkit")
) )
) )
@ -230,7 +245,8 @@ object AkkaBuild extends Build {
base = file("akka-osgi"), base = file("akka-osgi"),
dependencies = Seq(actor), dependencies = Seq(actor),
settings = defaultSettings ++ OSGi.osgi ++ Seq( settings = defaultSettings ++ OSGi.osgi ++ Seq(
libraryDependencies ++= Dependencies.osgi libraryDependencies ++= Dependencies.osgi,
parallelExecution in Test := false
) )
) )
@ -239,7 +255,8 @@ object AkkaBuild extends Build {
base = file("akka-osgi-aries"), base = file("akka-osgi-aries"),
dependencies = Seq(osgi % "compile;test->test"), dependencies = Seq(osgi % "compile;test->test"),
settings = defaultSettings ++ OSGi.osgiAries ++ Seq( settings = defaultSettings ++ OSGi.osgiAries ++ Seq(
libraryDependencies ++= Dependencies.osgiAries libraryDependencies ++= Dependencies.osgiAries,
parallelExecution in Test := false
) )
) )
@ -395,6 +412,7 @@ object AkkaBuild extends Build {
ivyLoggingLevel in ThisBuild := UpdateLogging.Quiet, ivyLoggingLevel in ThisBuild := UpdateLogging.Quiet,
parallelExecution in Test := System.getProperty("akka.parallelExecution", "false").toBoolean, parallelExecution in Test := System.getProperty("akka.parallelExecution", "false").toBoolean,
logBuffered in Test := System.getProperty("akka.logBufferedTests", "false").toBoolean,
excludeTestNames := useExcludeTestNames, excludeTestNames := useExcludeTestNames,
excludeTestTags := useExcludeTestTags, excludeTestTags := useExcludeTestTags,

16
repl
View file

@ -1,16 +0,0 @@
import akka.actor._
import akka.dispatch.{ Future, Promise }
import com.typesafe.config.ConfigFactory
import akka.testkit._
val remoteConfig = try {
Class.forName("akka.remote.RemoteActorRefProvider")
"\nakka.actor.provider=akka.remote.RemoteActorRefProvider"
} catch {
case _: ClassNotFoundException => ""
}
val config=ConfigFactory.parseString("akka.daemonic=on" + remoteConfig)
val sys=ActorSystem("repl", config.withFallback(ConfigFactory.load())).asInstanceOf[ExtendedActorSystem]
implicit val ec=sys.dispatcher
import akka.util.duration._
import akka.util.Timeout
implicit val timeout=Timeout(5 seconds)