#2292 - Removing akka.util.Duration etc and replace it with scala.concurrent.util.Duration

This commit is contained in:
Viktor Klang 2012-06-29 13:33:20 +02:00
parent 4f1caeefd4
commit 54a3a44bf8
200 changed files with 347 additions and 854 deletions

View file

@ -4,7 +4,7 @@ import akka.util.Timeout;
import akka.actor.ActorSystem; import akka.actor.ActorSystem;
import akka.japi.*; import akka.japi.*;
import akka.util.Duration; import scala.concurrent.util.Duration;
import akka.testkit.TestKitExtension; import akka.testkit.TestKitExtension;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.BeforeClass; import org.junit.BeforeClass;

View file

@ -4,6 +4,7 @@
package akka.util; package akka.util;
import org.junit.Test; import org.junit.Test;
import scala.concurrent.util.Duration;
public class JavaDuration { public class JavaDuration {

View file

@ -8,7 +8,7 @@ import language.postfixOps
import akka.testkit._ import akka.testkit._
import akka.testkit.DefaultTimeout import akka.testkit.DefaultTimeout
import akka.testkit.TestEvent._ import akka.testkit.TestEvent._
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.routing._ import akka.routing._
import org.scalatest.BeforeAndAfterEach import org.scalatest.BeforeAndAfterEach
import akka.ConfigurationException import akka.ConfigurationException

View file

@ -6,7 +6,7 @@ package akka.actor
import akka.testkit._ import akka.testkit._
import org.scalatest.BeforeAndAfterEach import org.scalatest.BeforeAndAfterEach
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.dispatch.Await import akka.dispatch.Await
import akka.pattern.ask import akka.pattern.ask
@ -87,7 +87,7 @@ class ActorFireForgetRequestReplySpec extends AkkaSpec with BeforeAndAfterEach w
actor.isTerminated must be(false) actor.isTerminated must be(false)
actor ! "Die" actor ! "Die"
state.finished.await state.finished.await
1.second.dilated.sleep() Thread.sleep(1.second.dilated.toMillis)
actor.isTerminated must be(true) actor.isTerminated must be(true)
system.stop(supervisor) system.stop(supervisor)
} }

View file

@ -11,7 +11,7 @@ import org.scalatest.matchers.MustMatchers
import akka.actor.Actor._ import akka.actor.Actor._
import akka.testkit._ import akka.testkit._
import akka.util.duration._ import scala.concurrent.util.duration._
import java.util.concurrent.atomic._ import java.util.concurrent.atomic._
import akka.dispatch.Await import akka.dispatch.Await
import akka.pattern.ask import akka.pattern.ask

View file

@ -6,7 +6,7 @@ package akka.actor
import language.postfixOps import language.postfixOps
import akka.testkit._ import akka.testkit._
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.dispatch.Await import akka.dispatch.Await
import akka.pattern.ask import akka.pattern.ask
import java.net.MalformedURLException import java.net.MalformedURLException

View file

@ -11,12 +11,13 @@ import org.scalatest.matchers.MustMatchers
import akka.testkit._ import akka.testkit._
import akka.util.Timeout import akka.util.Timeout
import akka.util.duration._ import scala.concurrent.util.duration._
import java.lang.IllegalStateException import java.lang.IllegalStateException
import java.util.concurrent.{ CountDownLatch, TimeUnit } import java.util.concurrent.{ CountDownLatch, TimeUnit }
import akka.dispatch.{ Await, DefaultPromise, Promise, Future } import akka.dispatch.{ Await, DefaultPromise, Promise, Future }
import akka.pattern.ask import akka.pattern.ask
import akka.serialization.JavaSerializer import akka.serialization.JavaSerializer
import akka.actor.NonPublicClass
object ActorRefSpec { object ActorRefSpec {
@ -54,7 +55,7 @@ object ActorRefSpec {
} }
private def work { private def work {
1.second.dilated.sleep Thread.sleep(1.second.dilated.toMillis)
} }
} }

View file

@ -9,7 +9,7 @@ import akka.testkit._
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import akka.dispatch.Await import akka.dispatch.Await
import akka.util.duration._ import scala.concurrent.util.duration._
import scala.collection.JavaConverters import scala.collection.JavaConverters
import java.util.concurrent.{ TimeUnit, RejectedExecutionException, CountDownLatch, ConcurrentLinkedQueue } import java.util.concurrent.{ TimeUnit, RejectedExecutionException, CountDownLatch, ConcurrentLinkedQueue }
import akka.pattern.ask import akka.pattern.ask
@ -105,7 +105,7 @@ class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExt
for (i 1 to count) { for (i 1 to count) {
system2.registerOnTermination { system2.registerOnTermination {
(i % 3).millis.dilated.sleep() Thread.sleep((i % 3).millis.dilated.toMillis)
result add i result add i
latch.countDown() latch.countDown()
} }
@ -127,7 +127,7 @@ class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExt
var callbackWasRun = false var callbackWasRun = false
system2.registerOnTermination { system2.registerOnTermination {
50.millis.dilated.sleep() Thread.sleep(50.millis.dilated.toMillis)
callbackWasRun = true callbackWasRun = true
} }

View file

@ -3,7 +3,7 @@
*/ */
package akka.actor package akka.actor
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.testkit._ import akka.testkit._
import akka.dispatch.Await import akka.dispatch.Await
import akka.util.Timeout import akka.util.Timeout

View file

@ -10,7 +10,7 @@ import akka.testkit.DefaultTimeout
import akka.testkit.TestEvent._ import akka.testkit.TestEvent._
import akka.dispatch.{ Await, BoundedDequeBasedMailbox } import akka.dispatch.{ Await, BoundedDequeBasedMailbox }
import akka.pattern.ask import akka.pattern.ask
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.actor.ActorSystem.Settings import akka.actor.ActorSystem.Settings
import com.typesafe.config.{ Config, ConfigFactory } import com.typesafe.config.{ Config, ConfigFactory }
import org.scalatest.BeforeAndAfterEach import org.scalatest.BeforeAndAfterEach

View file

@ -10,7 +10,7 @@ import akka.testkit.DefaultTimeout
import akka.testkit.TestEvent._ import akka.testkit.TestEvent._
import akka.dispatch.Await import akka.dispatch.Await
import akka.pattern.ask import akka.pattern.ask
import akka.util.duration._ import scala.concurrent.util.duration._
import com.typesafe.config.{ Config, ConfigFactory } import com.typesafe.config.{ Config, ConfigFactory }
import org.scalatest.BeforeAndAfterEach import org.scalatest.BeforeAndAfterEach
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite

View file

@ -4,7 +4,7 @@ import language.postfixOps
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.dispatch.UnboundedMailbox import akka.dispatch.UnboundedMailbox
import akka.util.duration._ import scala.concurrent.util.duration._
object ConsistencySpec { object ConsistencySpec {
val config = """ val config = """

View file

@ -7,7 +7,7 @@ package akka.actor
import language.postfixOps import language.postfixOps
import akka.testkit._ import akka.testkit._
import akka.util.duration._ import scala.concurrent.util.duration._
import java.util.concurrent.atomic._ import java.util.concurrent.atomic._
import akka.dispatch.Await import akka.dispatch.Await
import akka.pattern.ask import akka.pattern.ask

View file

@ -10,7 +10,7 @@ import akka.testkit.AkkaSpec
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import com.typesafe.config.ConfigParseOptions import com.typesafe.config.ConfigParseOptions
import akka.routing._ import akka.routing._
import akka.util.duration._ import scala.concurrent.util.duration._
object DeployerSpec { object DeployerSpec {
val deployerConf = ConfigFactory.parseString(""" val deployerConf = ConfigFactory.parseString("""

View file

@ -9,11 +9,12 @@ import language.postfixOps
import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach } import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach }
import akka.testkit._ import akka.testkit._
import TestEvent.Mute import TestEvent.Mute
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.event._ import akka.event._
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import akka.dispatch.Await import akka.dispatch.Await
import akka.util.{ Timeout, Duration } import akka.util.Timeout
import scala.concurrent.util.Duration
object FSMActorSpec { object FSMActorSpec {
val timeout = Timeout(2 seconds) val timeout = Timeout(2 seconds)

View file

@ -7,8 +7,8 @@ package akka.actor
import language.postfixOps import language.postfixOps
import akka.testkit._ import akka.testkit._
import akka.util.Duration import scala.concurrent.util.Duration
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.event.Logging import akka.event.Logging
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])

View file

@ -6,8 +6,8 @@ package akka.actor
import language.postfixOps import language.postfixOps
import akka.testkit._ import akka.testkit._
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.util.Duration import scala.concurrent.util.Duration
object FSMTransitionSpec { object FSMTransitionSpec {

View file

@ -7,9 +7,9 @@ package akka.actor
import language.postfixOps import language.postfixOps
import akka.testkit._ import akka.testkit._
import akka.util.duration._ import scala.concurrent.util.duration._
import Actor._ import Actor._
import akka.util.Duration import scala.concurrent.util.Duration
import akka.dispatch.Await import akka.dispatch.Await
import akka.pattern.ask import akka.pattern.ask

View file

@ -6,8 +6,9 @@ package akka.actor
import language.postfixOps import language.postfixOps
import akka.util.{ ByteString, Duration, Deadline } import akka.util.ByteString
import akka.util.duration._ import scala.concurrent.util.{ Duration, Deadline }
import scala.concurrent.util.duration._
import scala.util.continuations._ import scala.util.continuations._
import akka.testkit._ import akka.testkit._
import akka.dispatch.{ Await, Future, Promise, ExecutionContext, MessageDispatcher } import akka.dispatch.{ Await, Future, Promise, ExecutionContext, MessageDispatcher }

View file

@ -7,7 +7,7 @@ package akka.actor
import language.postfixOps import language.postfixOps
import akka.testkit._ import akka.testkit._
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.util.Timeout import akka.util.Timeout
import akka.dispatch.{ Await, Future } import akka.dispatch.{ Await, Future }

View file

@ -7,7 +7,7 @@ package akka.actor
import language.postfixOps import language.postfixOps
import akka.testkit._ import akka.testkit._
import akka.util.duration._ import scala.concurrent.util.duration._
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import akka.dispatch.Await import akka.dispatch.Await

View file

@ -15,8 +15,8 @@ import java.util.concurrent.{ TimeUnit, CountDownLatch }
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.testkit.DefaultTimeout import akka.testkit.DefaultTimeout
import akka.testkit.TestLatch import akka.testkit.TestLatch
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.util.Duration import scala.concurrent.util.Duration
import akka.pattern.ask import akka.pattern.ask
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])

View file

@ -3,7 +3,7 @@ package akka.actor
import language.postfixOps import language.postfixOps
import org.scalatest.BeforeAndAfterEach import org.scalatest.BeforeAndAfterEach
import akka.util.duration._ import scala.concurrent.util.duration._
import java.util.concurrent.{ CountDownLatch, ConcurrentLinkedQueue, TimeUnit } import java.util.concurrent.{ CountDownLatch, ConcurrentLinkedQueue, TimeUnit }
import akka.testkit._ import akka.testkit._
import akka.dispatch.Await import akka.dispatch.Await
@ -115,9 +115,9 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout
val timeout = collectCancellable(system.scheduler.schedule(initialDelay, delay) { val timeout = collectCancellable(system.scheduler.schedule(initialDelay, delay) {
ticks.incrementAndGet() ticks.incrementAndGet()
}) })
10.milliseconds.dilated.sleep() Thread.sleep(10.milliseconds.dilated.toMillis)
timeout.cancel() timeout.cancel()
(initialDelay + 100.milliseconds.dilated).sleep() Thread.sleep((initialDelay + 100.milliseconds.dilated).toMillis)
ticks.get must be(0) ticks.get must be(0)
} }
@ -130,9 +130,9 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout
val timeout = collectCancellable(system.scheduler.schedule(initialDelay, delay) { val timeout = collectCancellable(system.scheduler.schedule(initialDelay, delay) {
ticks.incrementAndGet() ticks.incrementAndGet()
}) })
(initialDelay + 100.milliseconds.dilated).sleep() Thread.sleep((initialDelay + 100.milliseconds.dilated).toMillis)
timeout.cancel() timeout.cancel()
(delay + 100.milliseconds.dilated).sleep() Thread.sleep((delay + 100.milliseconds.dilated).toMillis)
ticks.get must be(1) ticks.get must be(1)
} }

View file

@ -10,8 +10,8 @@ import akka.testkit._
import java.util.concurrent.{ TimeUnit, CountDownLatch } import java.util.concurrent.{ TimeUnit, CountDownLatch }
import akka.dispatch.Await import akka.dispatch.Await
import akka.pattern.ask import akka.pattern.ask
import akka.util.Duration import scala.concurrent.util.Duration
import akka.util.duration._ import scala.concurrent.util.duration._
object SupervisorHierarchySpec { object SupervisorHierarchySpec {
class FireWorkerException(msg: String) extends Exception(msg) class FireWorkerException(msg: String) extends Exception(msg)

View file

@ -11,7 +11,7 @@ import java.util.concurrent.{ TimeUnit, CountDownLatch }
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.testkit.DefaultTimeout import akka.testkit.DefaultTimeout
import akka.pattern.ask import akka.pattern.ask
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.util.NonFatal import akka.util.NonFatal
object SupervisorMiscSpec { object SupervisorMiscSpec {

View file

@ -7,7 +7,7 @@ package akka.actor
import language.postfixOps import language.postfixOps
import org.scalatest.BeforeAndAfterEach import org.scalatest.BeforeAndAfterEach
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.{ Die, Ping } import akka.{ Die, Ping }
import akka.testkit.TestEvent._ import akka.testkit.TestEvent._
import akka.testkit._ import akka.testkit._

View file

@ -7,7 +7,7 @@ import language.postfixOps
import org.scalatest.WordSpec import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.actor.Actor._ import akka.actor.Actor._
import akka.testkit.{ TestKit, EventFilter, filterEvents, filterException } import akka.testkit.{ TestKit, EventFilter, filterEvents, filterException }
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec

View file

@ -14,7 +14,7 @@ import akka.testkit.ImplicitSender
import akka.testkit.DefaultTimeout import akka.testkit.DefaultTimeout
import akka.dispatch.Await import akka.dispatch.Await
import akka.pattern.ask import akka.pattern.ask
import akka.util.duration._ import scala.concurrent.util.duration._
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class Ticket669Spec extends AkkaSpec with BeforeAndAfterAll with ImplicitSender with DefaultTimeout { class Ticket669Spec extends AkkaSpec with BeforeAndAfterAll with ImplicitSender with DefaultTimeout {

View file

@ -6,9 +6,9 @@ package akka.actor
import language.postfixOps import language.postfixOps
import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach } import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach }
import akka.util.Duration import scala.concurrent.util.Duration
import akka.util.Timeout import akka.util.Timeout
import akka.util.duration._ import scala.concurrent.util.duration._
import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicReference
import annotation.tailrec import annotation.tailrec
import akka.testkit.{ EventFilter, filterEvents, AkkaSpec } import akka.testkit.{ EventFilter, filterEvents, AkkaSpec }

View file

@ -20,9 +20,10 @@ import akka.dispatch._
import akka.event.Logging.Error import akka.event.Logging.Error
import akka.pattern.ask import akka.pattern.ask
import akka.testkit._ import akka.testkit._
import akka.util.{ Timeout, Switch, Duration } import akka.util.{ Timeout, Switch }
import akka.util.duration._ import scala.concurrent.util.duration._
import annotation.tailrec import scala.concurrent.util.Duration
import scala.annotation.tailrec
object ActorModelSpec { object ActorModelSpec {

View file

@ -6,8 +6,8 @@ import java.util.concurrent.{ CountDownLatch, TimeUnit }
import java.util.concurrent.atomic.{ AtomicBoolean, AtomicInteger } import java.util.concurrent.atomic.{ AtomicBoolean, AtomicInteger }
import akka.testkit.{ filterEvents, EventFilter, AkkaSpec } import akka.testkit.{ filterEvents, EventFilter, AkkaSpec }
import akka.actor.{ Props, Actor } import akka.actor.{ Props, Actor }
import akka.util.Duration import scala.concurrent.util.Duration
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.testkit.DefaultTimeout import akka.testkit.DefaultTimeout
import akka.dispatch.{ Await, PinnedDispatcher, Dispatchers, Dispatcher } import akka.dispatch.{ Await, PinnedDispatcher, Dispatchers, Dispatcher }
import akka.pattern.ask import akka.pattern.ask

View file

@ -14,7 +14,7 @@ import scala.collection.JavaConverters._
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import akka.actor.Actor import akka.actor.Actor
import akka.actor.Props import akka.actor.Props
import akka.util.duration._ import scala.concurrent.util.duration._
object DispatchersSpec { object DispatchersSpec {
val config = """ val config = """

View file

@ -9,8 +9,8 @@ import language.postfixOps
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.util.Duration import scala.concurrent.util.Duration
import akka.actor.ActorSystem import akka.actor.ActorSystem
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])

View file

@ -7,7 +7,7 @@ import language.postfixOps
import akka.actor.{ Actor, Props } import akka.actor.{ Actor, Props }
import akka.dispatch.{ Future, Await } import akka.dispatch.{ Future, Await }
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.testkit.DefaultTimeout import akka.testkit.DefaultTimeout
import akka.pattern.{ ask, pipe } import akka.pattern.{ ask, pipe }

View file

@ -10,7 +10,7 @@ import org.scalacheck.Prop._
import org.scalacheck.Gen._ import org.scalacheck.Gen._
import akka.actor._ import akka.actor._
import akka.testkit.{ EventFilter, filterEvents, filterException } import akka.testkit.{ EventFilter, filterEvents, filterException }
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import akka.testkit.DefaultTimeout import akka.testkit.DefaultTimeout

View file

@ -8,7 +8,7 @@ import language.postfixOps
import org.scalatest.BeforeAndAfterEach import org.scalatest.BeforeAndAfterEach
import akka.testkit._ import akka.testkit._
import akka.util.duration._ import scala.concurrent.util.duration._
import java.util.concurrent.atomic._ import java.util.concurrent.atomic._
import akka.actor.{ Props, Actor, ActorRef, ActorSystem } import akka.actor.{ Props, Actor, ActorRef, ActorSystem }
import java.util.Comparator import java.util.Comparator

View file

@ -6,7 +6,7 @@ package akka.event
import language.postfixOps import language.postfixOps
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.actor.{ Actor, ActorRef, ActorSystemImpl, ActorSystem, Props, UnhandledMessage } import akka.actor.{ Actor, ActorRef, ActorSystemImpl, ActorSystem, Props, UnhandledMessage }
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._

View file

@ -6,10 +6,10 @@ package akka.event
import language.postfixOps import language.postfixOps
import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach } import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach }
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.testkit._ import akka.testkit._
import org.scalatest.WordSpec import org.scalatest.WordSpec
import akka.util.Duration import scala.concurrent.util.Duration
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
import java.util.Properties import java.util.Properties

View file

@ -6,7 +6,7 @@ package akka.pattern
import language.postfixOps import language.postfixOps
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.testkit.DefaultTimeout import akka.testkit.DefaultTimeout
class AskSpec extends AkkaSpec with DefaultTimeout { class AskSpec extends AkkaSpec with DefaultTimeout {

View file

@ -4,7 +4,7 @@
package akka.pattern package akka.pattern
import akka.testkit._ import akka.testkit._
import akka.util.duration._ import scala.concurrent.util.duration._
import org.scalatest.BeforeAndAfter import org.scalatest.BeforeAndAfter
import akka.dispatch.{ Promise, Await, Future } import akka.dispatch.{ Promise, Await, Future }

View file

@ -6,7 +6,7 @@ package akka.pattern
import language.postfixOps import language.postfixOps
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.testkit._ import akka.testkit._
import org.scalatest.BeforeAndAfter import org.scalatest.BeforeAndAfter
import akka.dispatch.Future import akka.dispatch.Future
@ -139,7 +139,7 @@ class CircuitBreakerSpec extends AkkaSpec with BeforeAndAfter {
"increment failure count on callTimeout" in { "increment failure count on callTimeout" in {
breakers.shortCallTimeoutCb.withSyncCircuitBreaker({ breakers.shortCallTimeoutCb.withSyncCircuitBreaker({
100.millis.dilated.sleep() Thread.sleep(100.millis.dilated.toMillis)
}) })
breakers.shortCallTimeoutCb.currentFailureCount must be(1) breakers.shortCallTimeoutCb.currentFailureCount must be(1)
} }
@ -231,7 +231,7 @@ class CircuitBreakerSpec extends AkkaSpec with BeforeAndAfter {
"increment failure count on callTimeout" in { "increment failure count on callTimeout" in {
breakers.shortCallTimeoutCb.withCircuitBreaker { breakers.shortCallTimeoutCb.withCircuitBreaker {
Future { Future {
100.millis.dilated.sleep() Thread.sleep(100.millis.dilated.toMillis)
sayHi sayHi
} }
} }

View file

@ -9,15 +9,15 @@ import language.postfixOps
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.actor.Props import akka.actor.Props
import akka.actor.Actor import akka.actor.Actor
import akka.util.Duration import scala.concurrent.util.Duration
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.dispatch.{ Future, Promise, Await } import akka.dispatch.{ Future, Promise, Await }
object PatternSpec { object PatternSpec {
case class Work(duration: Duration) case class Work(duration: Duration)
class TargetActor extends Actor { class TargetActor extends Actor {
def receive = { def receive = {
case Work(duration) duration.sleep() case Work(duration) Thread.sleep(duration.toMillis)
} }
} }
} }

View file

@ -4,8 +4,8 @@ import akka.performance.workbench.PerformanceSpec
import akka.actor._ import akka.actor._
import java.util.concurrent.{ ThreadPoolExecutor, CountDownLatch, TimeUnit } import java.util.concurrent.{ ThreadPoolExecutor, CountDownLatch, TimeUnit }
import akka.dispatch._ import akka.dispatch._
import akka.util.Duration import scala.concurrent.util.Duration
import akka.util.duration._ import scala.concurrent.util.duration._
// -server -Xms512M -Xmx1024M -XX:+UseParallelGC -Dbenchmark=true -Dbenchmark.repeatFactor=500 // -server -Xms512M -Xmx1024M -XX:+UseParallelGC -Dbenchmark=true -Dbenchmark.repeatFactor=500
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])

View file

@ -4,8 +4,8 @@ import akka.performance.workbench.PerformanceSpec
import akka.actor._ import akka.actor._
import java.util.concurrent.{ ThreadPoolExecutor, CountDownLatch, TimeUnit } import java.util.concurrent.{ ThreadPoolExecutor, CountDownLatch, TimeUnit }
import akka.dispatch._ import akka.dispatch._
import akka.util.Duration import scala.concurrent.util.Duration
import akka.util.duration._ import scala.concurrent.util.duration._
// -server -Xms512M -Xmx1024M -XX:+UseParallelGC -Dbenchmark=true -Dbenchmark.repeatFactor=500 // -server -Xms512M -Xmx1024M -XX:+UseParallelGC -Dbenchmark=true -Dbenchmark.repeatFactor=500
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])

View file

@ -4,7 +4,7 @@ import scala.collection.immutable.TreeMap
import org.apache.commons.math.stat.descriptive.DescriptiveStatistics import org.apache.commons.math.stat.descriptive.DescriptiveStatistics
import org.scalatest.BeforeAndAfterEach import org.scalatest.BeforeAndAfterEach
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.util.Duration import scala.concurrent.util.Duration
import com.typesafe.config.Config import com.typesafe.config.Config
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import akka.event.Logging import akka.event.Logging

View file

@ -9,11 +9,11 @@ import akka.actor.Actor
import akka.testkit._ import akka.testkit._
import akka.actor.Props import akka.actor.Props
import akka.dispatch.Await import akka.dispatch.Await
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.actor.ActorRef import akka.actor.ActorRef
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import akka.pattern.ask import akka.pattern.ask
import akka.util.Duration import scala.concurrent.util.Duration
import java.util.concurrent.TimeoutException import java.util.concurrent.TimeoutException
object ResizerSpec { object ResizerSpec {
@ -174,7 +174,7 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with
val router = system.actorOf(Props(new Actor { val router = system.actorOf(Props(new Actor {
def receive = { def receive = {
case d: Duration d.dilated.sleep; sender ! "done" case d: Duration Thread.sleep(d.dilated.toMillis); sender ! "done"
case "echo" sender ! "reply" case "echo" sender ! "reply"
} }
}).withRouter(RoundRobinRouter(resizer = Some(resizer)))) }).withRouter(RoundRobinRouter(resizer = Some(resizer))))
@ -221,26 +221,25 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with
val router = system.actorOf(Props(new Actor { val router = system.actorOf(Props(new Actor {
def receive = { def receive = {
case n: Int case n: Int Thread.sleep((n millis).dilated.toMillis)
(n millis).dilated.sleep
} }
}).withRouter(RoundRobinRouter(resizer = Some(resizer)))) }).withRouter(RoundRobinRouter(resizer = Some(resizer))))
// put some pressure on the router // put some pressure on the router
for (m 0 to 5) { for (m 0 to 5) {
router ! 100 router ! 100
(5 millis).dilated.sleep Thread.sleep((5 millis).dilated.toMillis)
} }
val z = Await.result(router ? CurrentRoutees, 5 seconds).asInstanceOf[RouterRoutees].routees.size val z = Await.result(router ? CurrentRoutees, 5 seconds).asInstanceOf[RouterRoutees].routees.size
z must be >= (2) z must be >= (2)
(300 millis).dilated.sleep Thread.sleep((300 millis).dilated.toMillis)
// let it cool down // let it cool down
for (m 0 to 5) { for (m 0 to 5) {
router ! 1 router ! 1
(500 millis).dilated.sleep Thread.sleep((500 millis).dilated.toMillis)
} }
awaitCond( awaitCond(

View file

@ -9,9 +9,9 @@ import java.util.concurrent.atomic.AtomicInteger
import akka.actor._ import akka.actor._
import scala.collection.mutable.LinkedList import scala.collection.mutable.LinkedList
import akka.testkit._ import akka.testkit._
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.dispatch.Await import akka.dispatch.Await
import akka.util.Duration import scala.concurrent.util.Duration
import akka.ConfigurationException import akka.ConfigurationException
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import akka.pattern.{ ask, pipe } import akka.pattern.{ ask, pipe }

View file

@ -11,7 +11,7 @@ import akka.actor._
import java.io._ import java.io._
import akka.dispatch.Await import akka.dispatch.Await
import akka.util.Timeout import akka.util.Timeout
import akka.util.duration._ import scala.concurrent.util.duration._
import scala.reflect.BeanInfo import scala.reflect.BeanInfo
import com.google.protobuf.Message import com.google.protobuf.Message
import akka.pattern.ask import akka.pattern.ask

View file

@ -7,7 +7,9 @@ import language.postfixOps
import org.scalatest.WordSpec import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import duration._ import scala.concurrent.util.Duration
import scala.concurrent.util.duration._
import java.util.concurrent.TimeUnit._ import java.util.concurrent.TimeUnit._
class DurationSpec extends WordSpec with MustMatchers { class DurationSpec extends WordSpec with MustMatchers {
@ -15,8 +17,8 @@ class DurationSpec extends WordSpec with MustMatchers {
"Duration" must { "Duration" must {
"form a one-dimensional vector field" in { "form a one-dimensional vector field" in {
val zero = 0.seconds val zero = 0 seconds
val one = 1.second val one = 1 second
val two = one + one val two = one + one
val three = 3 * one val three = 3 * one
(0 * one) must be(zero) (0 * one) must be(zero)
@ -53,7 +55,7 @@ class DurationSpec extends WordSpec with MustMatchers {
assert(minf != one) assert(minf != one)
} }
"check its range" in { /*"check its range" in {
for (unit Seq(DAYS, HOURS, MINUTES, SECONDS, MILLISECONDS, MICROSECONDS, NANOSECONDS)) { for (unit Seq(DAYS, HOURS, MINUTES, SECONDS, MILLISECONDS, MICROSECONDS, NANOSECONDS)) {
val x = unit.convert(Long.MaxValue, NANOSECONDS) val x = unit.convert(Long.MaxValue, NANOSECONDS)
val dur = Duration(x, unit) val dur = Duration(x, unit)
@ -80,7 +82,7 @@ class DurationSpec extends WordSpec with MustMatchers {
intercept[IllegalArgumentException] { Duration("%.0f".format(x + 10000000d) + unit.toString.toLowerCase) } intercept[IllegalArgumentException] { Duration("%.0f".format(x + 10000000d) + unit.toString.toLowerCase) }
intercept[IllegalArgumentException] { Duration("-%.0f".format(x + 10000000d) + unit.toString.toLowerCase) } intercept[IllegalArgumentException] { Duration("-%.0f".format(x + 10000000d) + unit.toString.toLowerCase) }
} }
} }*/
"support fromNow" in { "support fromNow" in {
val dead = 2.seconds.fromNow val dead = 2.seconds.fromNow
@ -88,7 +90,7 @@ class DurationSpec extends WordSpec with MustMatchers {
// view bounds vs. very local type inference vs. operator precedence: sigh // view bounds vs. very local type inference vs. operator precedence: sigh
dead.timeLeft must be > (1 second: Duration) dead.timeLeft must be > (1 second: Duration)
dead2.timeLeft must be > (1 second: Duration) dead2.timeLeft must be > (1 second: Duration)
1.second.sleep Thread.sleep(1.second.toMillis)
dead.timeLeft must be < (1 second: Duration) dead.timeLeft must be < (1 second: Duration)
dead2.timeLeft must be < (1 second: Duration) dead2.timeLeft must be < (1 second: Duration)
} }

View file

@ -16,7 +16,7 @@
package akka.util.internal; package akka.util.internal;
import akka.event.LoggingAdapter; import akka.event.LoggingAdapter;
import akka.util.Duration; import scala.concurrent.util.Duration;
import java.util.*; import java.util.*;
import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadFactory;

View file

@ -15,7 +15,7 @@
*/ */
package akka.util.internal; package akka.util.internal;
import akka.util.Duration; import scala.concurrent.util.Duration;
import java.util.Set; import java.util.Set;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;

View file

@ -15,9 +15,9 @@ import akka.japi.Procedure
import java.io.{ NotSerializableException, ObjectOutputStream } import java.io.{ NotSerializableException, ObjectOutputStream }
import akka.serialization.SerializationExtension import akka.serialization.SerializationExtension
import akka.event.Logging.LogEventException import akka.event.Logging.LogEventException
import collection.immutable.{ TreeSet, TreeMap } import scala.collection.immutable.{ TreeSet, TreeMap }
import akka.util.{ Unsafe, Duration, Helpers, NonFatal } import akka.util.{ Unsafe, Helpers, NonFatal }
import java.util.concurrent.atomic.AtomicLong import scala.concurrent.util.Duration
//TODO: everything here for current compatibility - could be limited more //TODO: everything here for current compatibility - could be limited more

View file

@ -9,6 +9,7 @@ import akka.dispatch._
import akka.pattern.ask import akka.pattern.ask
import com.typesafe.config.{ Config, ConfigFactory } import com.typesafe.config.{ Config, ConfigFactory }
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.concurrent.util.Duration
import java.io.Closeable import java.io.Closeable
import akka.dispatch.Await.{ Awaitable, CanAwait } import akka.dispatch.Await.{ Awaitable, CanAwait }
import akka.util._ import akka.util._

View file

@ -4,7 +4,7 @@
package akka.actor package akka.actor
import akka.util.Duration import scala.concurrent.util.Duration
import com.typesafe.config._ import com.typesafe.config._
import akka.routing._ import akka.routing._
import java.util.concurrent.{ TimeUnit } import java.util.concurrent.{ TimeUnit }

View file

@ -6,7 +6,7 @@ package akka.actor
import language.implicitConversions import language.implicitConversions
import akka.util._ import akka.util._
import scala.concurrent.util.Duration
import scala.collection.mutable import scala.collection.mutable
import akka.routing.{ Deafen, Listen, Listeners } import akka.routing.{ Deafen, Listen, Listeners }

View file

@ -9,8 +9,7 @@ import java.util.concurrent.TimeUnit
import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.ArrayBuffer
import scala.collection.JavaConversions._ import scala.collection.JavaConversions._
import java.lang.{ Iterable JIterable } import java.lang.{ Iterable JIterable }
import akka.util.Duration import scala.concurrent.util.Duration
/** /**
* INTERNAL API * INTERNAL API
*/ */

View file

@ -7,7 +7,8 @@ import language.higherKinds
import language.postfixOps import language.postfixOps
import akka.dispatch.{ Future, ExecutionContext } import akka.dispatch.{ Future, ExecutionContext }
import akka.util.{ ByteString, Duration, NonFatal } import scala.concurrent.util.Duration
import akka.util.{ ByteString, NonFatal }
import java.net.{ SocketAddress, InetSocketAddress } import java.net.{ SocketAddress, InetSocketAddress }
import java.io.IOException import java.io.IOException
import java.nio.ByteBuffer import java.nio.ByteBuffer

View file

@ -4,7 +4,7 @@
package akka.actor package akka.actor
import akka.util.Duration import scala.concurrent.util.Duration
import akka.util.internal.{ TimerTask, HashedWheelTimer, Timeout HWTimeout, Timer } import akka.util.internal.{ TimerTask, HashedWheelTimer, Timeout HWTimeout, Timer }
import akka.event.LoggingAdapter import akka.event.LoggingAdapter
import akka.dispatch.MessageDispatcher import akka.dispatch.MessageDispatcher

View file

@ -7,7 +7,8 @@ import language.existentials
import akka.japi.{ Creator, Option JOption } import akka.japi.{ Creator, Option JOption }
import java.lang.reflect.{ InvocationTargetException, Method, InvocationHandler, Proxy } import java.lang.reflect.{ InvocationTargetException, Method, InvocationHandler, Proxy }
import akka.util.{ Timeout, NonFatal, Duration } import akka.util.{ Timeout, NonFatal }
import scala.concurrent.util.Duration
import akka.util.Reflect.instantiator import akka.util.Reflect.instantiator
import java.util.concurrent.atomic.{ AtomicReference AtomVar } import java.util.concurrent.atomic.{ AtomicReference AtomVar }
import akka.dispatch._ import akka.dispatch._

View file

@ -14,7 +14,8 @@ import com.typesafe.config.Config
import akka.serialization.SerializationExtension import akka.serialization.SerializationExtension
import akka.event.Logging.LogEventException import akka.event.Logging.LogEventException
import scala.concurrent.forkjoin.{ ForkJoinTask, ForkJoinPool } import scala.concurrent.forkjoin.{ ForkJoinTask, ForkJoinPool }
import akka.util.{ Unsafe, Duration, NonFatal, Index } import akka.util.{ Unsafe, NonFatal, Index }
import scala.concurrent.util.Duration
final case class Envelope private (val message: Any, val sender: ActorRef) final case class Envelope private (val message: Any, val sender: ActorRef)

View file

@ -5,8 +5,9 @@
package akka.dispatch package akka.dispatch
import akka.actor.{ ActorCell, ActorRef } import akka.actor.{ ActorCell, ActorRef }
import annotation.tailrec import scala.annotation.tailrec
import akka.util.{ Duration, Helpers } import scala.concurrent.util.Duration
import akka.util.Helpers
import java.util.{ Comparator, Iterator } import java.util.{ Comparator, Iterator }
import java.util.concurrent.{ Executor, LinkedBlockingQueue, ConcurrentLinkedQueue, ConcurrentSkipListSet } import java.util.concurrent.{ Executor, LinkedBlockingQueue, ConcurrentLinkedQueue, ConcurrentSkipListSet }
import akka.actor.ActorSystemImpl import akka.actor.ActorSystemImpl

View file

@ -7,7 +7,7 @@ package akka.dispatch
import akka.event.Logging.Error import akka.event.Logging.Error
import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicReference
import akka.actor.ActorCell import akka.actor.ActorCell
import akka.util.Duration import scala.concurrent.util.Duration
import java.util.concurrent._ import java.util.concurrent._
import akka.event.Logging import akka.event.Logging

View file

@ -9,7 +9,7 @@ import com.typesafe.config.{ ConfigFactory, Config }
import akka.actor.{ Scheduler, DynamicAccess, ActorSystem } import akka.actor.{ Scheduler, DynamicAccess, ActorSystem }
import akka.event.Logging.Warning import akka.event.Logging.Warning
import akka.event.EventStream import akka.event.EventStream
import akka.util.Duration import scala.concurrent.util.Duration
/** /**
* DispatcherPrerequisites represents useful contextual pieces when constructing a MessageDispatcher * DispatcherPrerequisites represents useful contextual pieces when constructing a MessageDispatcher

View file

@ -8,8 +8,6 @@ import language.implicitConversions
import language.postfixOps import language.postfixOps
import language.higherKinds import language.higherKinds
import akka.event.Logging.Error
import scala.Option
import akka.japi.{ Function JFunc, Option JOption } import akka.japi.{ Function JFunc, Option JOption }
import scala.util.continuations._ import scala.util.continuations._
import scala.reflect.ClassTag import scala.reflect.ClassTag
@ -17,16 +15,16 @@ import java.lang.{ Iterable ⇒ JIterable }
import java.util.{ LinkedList JLinkedList } import java.util.{ LinkedList JLinkedList }
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.collection.mutable.Stack import scala.collection.mutable.Stack
import akka.util.{ Duration, BoxedType } import akka.util.BoxedType
import akka.dispatch.Await.CanAwait import akka.dispatch.Await.CanAwait
import akka.util.NonFatal import akka.util.NonFatal
import akka.event.Logging.LogEventException import akka.event.Logging.{ LogEventException, Debug, Error }
import akka.event.Logging.Debug
import java.util.concurrent.TimeUnit.NANOSECONDS import java.util.concurrent.TimeUnit.NANOSECONDS
import java.util.concurrent.{ ExecutionException, Callable, TimeoutException } import java.util.concurrent.{ ExecutionException, Callable, TimeoutException }
import java.util.concurrent.atomic.{ AtomicInteger } import java.util.concurrent.atomic.{ AtomicInteger }
import akka.pattern.AskTimeoutException import akka.pattern.AskTimeoutException
import scala.util.DynamicVariable import scala.util.DynamicVariable
import scala.concurrent.util.Duration
import scala.runtime.{ BoxedUnit, AbstractPartialFunction } import scala.runtime.{ BoxedUnit, AbstractPartialFunction }
object Await { object Await {

View file

@ -5,8 +5,9 @@ package akka.dispatch
import akka.AkkaException import akka.AkkaException
import java.util.{ Comparator, PriorityQueue, Queue, Deque } import java.util.{ Comparator, PriorityQueue, Queue, Deque }
import akka.util._ import scala.concurrent.util.Duration
import akka.actor.{ ActorCell, ActorRef, Cell } import akka.actor.{ ActorCell, ActorRef, Cell }
import akka.util.{ Unsafe, NonFatal, BoundedBlockingQueue }
import java.util.concurrent._ import java.util.concurrent._
import annotation.tailrec import annotation.tailrec
import akka.event.Logging.Error import akka.event.Logging.Error

View file

@ -5,7 +5,7 @@
package akka.dispatch package akka.dispatch
import akka.actor.ActorCell import akka.actor.ActorCell
import akka.util.Duration import scala.concurrent.util.Duration
/** /**
* Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue. * Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue.

View file

@ -5,7 +5,7 @@
package akka.dispatch package akka.dispatch
import java.util.Collection import java.util.Collection
import akka.util.Duration import scala.concurrent.util.Duration
import scala.concurrent.forkjoin._ import scala.concurrent.forkjoin._
import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.atomic.AtomicLong
import java.util.concurrent.ArrayBlockingQueue import java.util.concurrent.ArrayBlockingQueue

View file

@ -9,7 +9,7 @@ import akka.actor._
import akka.{ ConfigurationException, AkkaException } import akka.{ ConfigurationException, AkkaException }
import akka.actor.ActorSystem.Settings import akka.actor.ActorSystem.Settings
import akka.util.{ Timeout, ReentrantGuard } import akka.util.{ Timeout, ReentrantGuard }
import akka.util.duration._ import scala.concurrent.util.duration._
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import java.util.concurrent.TimeoutException import java.util.concurrent.TimeoutException

View file

@ -7,8 +7,9 @@ import java.util.concurrent.atomic.{ AtomicInteger, AtomicLong, AtomicBoolean }
import akka.AkkaException import akka.AkkaException
import akka.actor.Scheduler import akka.actor.Scheduler
import akka.dispatch.{ Future, ExecutionContext, Await, Promise } import akka.dispatch.{ Future, ExecutionContext, Await, Promise }
import akka.util.{ Deadline, Duration, NonFatal, Unsafe } import akka.util.{ NonFatal, Unsafe }
import akka.util.duration._ import scala.concurrent.util.duration._
import scala.concurrent.util.{ Duration, Deadline }
import util.control.NoStackTrace import util.control.NoStackTrace
import java.util.concurrent.{ Callable, CopyOnWriteArrayList } import java.util.concurrent.{ Callable, CopyOnWriteArrayList }

View file

@ -4,7 +4,7 @@ package akka.pattern
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/ */
import akka.util.Duration import scala.concurrent.util.Duration
import akka.actor._ import akka.actor._
import akka.dispatch.{ ExecutionContext, Promise, Future } import akka.dispatch.{ ExecutionContext, Promise, Future }

View file

@ -5,7 +5,8 @@
package akka.pattern package akka.pattern
import akka.actor._ import akka.actor._
import akka.util.{ Timeout, Duration } import akka.util.{ Timeout }
import scala.concurrent.util.Duration
import akka.dispatch.{ Unwatch, Watch, Promise, Future } import akka.dispatch.{ Unwatch, Watch, Promise, Future }
trait GracefulStopSupport { trait GracefulStopSupport {

View file

@ -11,7 +11,8 @@ object Patterns {
import akka.actor.{ ActorRef, ActorSystem } import akka.actor.{ ActorRef, ActorSystem }
import akka.dispatch.Future import akka.dispatch.Future
import akka.pattern.{ ask scalaAsk, pipe scalaPipe, gracefulStop scalaGracefulStop, after scalaAfter } import akka.pattern.{ ask scalaAsk, pipe scalaPipe, gracefulStop scalaGracefulStop, after scalaAfter }
import akka.util.{ Timeout, Duration } import akka.util.Timeout
import scala.concurrent.util.Duration
/** /**
* <i>Java API for `akka.pattern.ask`:</i> * <i>Java API for `akka.pattern.ask`:</i>

View file

@ -4,8 +4,6 @@
package akka package akka
import akka.actor._ import akka.actor._
import akka.dispatch.{ Future, Promise }
import akka.util.{ Timeout, Duration }
/** /**
* == Commonly Used Patterns With Akka == * == Commonly Used Patterns With Akka ==

View file

@ -7,8 +7,8 @@ import language.implicitConversions
import language.postfixOps import language.postfixOps
import akka.actor._ import akka.actor._
import akka.util.Duration import scala.concurrent.util.Duration
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.ConfigurationException import akka.ConfigurationException
import akka.pattern.pipe import akka.pattern.pipe
import com.typesafe.config.Config import com.typesafe.config.Config

View file

@ -7,450 +7,8 @@ package akka.util
import language.implicitConversions import language.implicitConversions
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import TimeUnit._
import java.lang.{ Double JDouble } import java.lang.{ Double JDouble }
import scala.concurrent.util.Duration
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
case class Deadline private (time: Duration) extends Ordered[Deadline] {
def +(other: Duration): Deadline = copy(time = time + other)
def -(other: Duration): Deadline = copy(time = time - other)
def -(other: Deadline): Duration = time - other.time
def timeLeft: Duration = this - Deadline.now
def hasTimeLeft(): Boolean = !isOverdue() //Code reuse FTW
def isOverdue(): Boolean = (time.toNanos - System.nanoTime()) < 0
def compare(that: Deadline) = this.time compare that.time
}
object Deadline {
def now: Deadline = Deadline(Duration(System.nanoTime, NANOSECONDS))
implicit object DeadlineIsOrdered extends Ordering[Deadline] {
def compare(a: Deadline, b: Deadline) = a compare b
}
}
object Duration {
implicit def timeLeft(implicit d: Deadline): Duration = d.timeLeft
def apply(length: Long, unit: TimeUnit): FiniteDuration = new FiniteDuration(length, unit)
def apply(length: Double, unit: TimeUnit): FiniteDuration = fromNanos(unit.toNanos(1) * length)
def apply(length: Long, unit: String): FiniteDuration = new FiniteDuration(length, timeUnit(unit))
def fromNanos(nanos: Long): FiniteDuration = {
if (nanos % 86400000000000L == 0) {
Duration(nanos / 86400000000000L, DAYS)
} else if (nanos % 3600000000000L == 0) {
Duration(nanos / 3600000000000L, HOURS)
} else if (nanos % 60000000000L == 0) {
Duration(nanos / 60000000000L, MINUTES)
} else if (nanos % 1000000000L == 0) {
Duration(nanos / 1000000000L, SECONDS)
} else if (nanos % 1000000L == 0) {
Duration(nanos / 1000000L, MILLISECONDS)
} else if (nanos % 1000L == 0) {
Duration(nanos / 1000L, MICROSECONDS)
} else {
Duration(nanos, NANOSECONDS)
}
}
def fromNanos(nanos: Double): FiniteDuration = {
if (nanos > Long.MaxValue || nanos < Long.MinValue)
throw new IllegalArgumentException("trying to construct too large duration with " + nanos + "ns")
fromNanos((nanos + 0.5).asInstanceOf[Long])
}
/**
* Construct a Duration by parsing a String. In case of a format error, a
* RuntimeException is thrown. See `unapply(String)` for more information.
*/
def apply(s: String): Duration = unapply(s) getOrElse sys.error("format error")
/**
* Deconstruct a Duration into length and unit if it is finite.
*/
def unapply(d: Duration): Option[(Long, TimeUnit)] = {
if (d.finite_?) {
Some((d.length, d.unit))
} else {
None
}
}
private val RE = ("""^\s*(-?\d+(?:\.\d+)?)\s*""" + // length part
"(?:" + // units are distinguished in separate match groups
"(d|day|days)|" +
"(h|hour|hours)|" +
"(min|minute|minutes)|" +
"(s|sec|second|seconds)|" +
"(ms|milli|millis|millisecond|milliseconds)|" +
"(µs|micro|micros|microsecond|microseconds)|" +
"(ns|nano|nanos|nanosecond|nanoseconds)" +
""")\s*$""").r // close the non-capturing group
private val REinf = """^\s*Inf\s*$""".r
private val REminf = """^\s*(?:-\s*|Minus)Inf\s*""".r
/**
* Parse String, return None if no match. Format is `"<length><unit>"`, where
* whitespace is allowed before, between and after the parts. Infinities are
* designated by `"Inf"` and `"-Inf"` or `"MinusInf"`.
*/
def unapply(s: String): Option[Duration] = s match {
case RE(length, d, h, m, s, ms, mus, ns)
if (d ne null) Some(Duration(JDouble.parseDouble(length), DAYS)) else if (h ne null) Some(Duration(JDouble.parseDouble(length), HOURS)) else if (m ne null) Some(Duration(JDouble.parseDouble(length), MINUTES)) else if (s ne null) Some(Duration(JDouble.parseDouble(length), SECONDS)) else if (ms ne null) Some(Duration(JDouble.parseDouble(length), MILLISECONDS)) else if (mus ne null) Some(Duration(JDouble.parseDouble(length), MICROSECONDS)) else if (ns ne null) Some(Duration(JDouble.parseDouble(length), NANOSECONDS)) else
sys.error("made some error in regex (should not be possible)")
case REinf() Some(Inf)
case REminf() Some(MinusInf)
case _ None
}
/**
* Parse TimeUnit from string representation.
*/
def timeUnit(unit: String) = unit.toLowerCase match {
case "d" | "day" | "days" DAYS
case "h" | "hour" | "hours" HOURS
case "min" | "minute" | "minutes" MINUTES
case "s" | "sec" | "second" | "seconds" SECONDS
case "ms" | "milli" | "millis" | "millisecond" | "milliseconds" MILLISECONDS
case "µs" | "micro" | "micros" | "microsecond" | "microseconds" MICROSECONDS
case "ns" | "nano" | "nanos" | "nanosecond" | "nanoseconds" NANOSECONDS
}
val Zero: FiniteDuration = new FiniteDuration(0, NANOSECONDS)
val Undefined: Duration = new Duration with Infinite {
override def toString = "Duration.Undefined"
override def equals(other: Any) = other.asInstanceOf[AnyRef] eq this
override def +(other: Duration): Duration = throw new IllegalArgumentException("cannot add Undefined duration")
override def -(other: Duration): Duration = throw new IllegalArgumentException("cannot subtract Undefined duration")
override def *(factor: Double): Duration = throw new IllegalArgumentException("cannot multiply Undefined duration")
override def /(factor: Double): Duration = throw new IllegalArgumentException("cannot divide Undefined duration")
override def /(other: Duration): Double = throw new IllegalArgumentException("cannot divide Undefined duration")
def compare(other: Duration) = throw new IllegalArgumentException("cannot compare Undefined duration")
def unary_- : Duration = throw new IllegalArgumentException("cannot negate Undefined duration")
}
trait Infinite {
this: Duration
def +(other: Duration): Duration =
other match {
case _: this.type this
case _: Infinite throw new IllegalArgumentException("illegal addition of infinities")
case _ this
}
def -(other: Duration): Duration =
other match {
case _: this.type throw new IllegalArgumentException("illegal subtraction of infinities")
case _ this
}
def *(factor: Double): Duration = this
def /(factor: Double): Duration = this
def /(other: Duration): Double =
other match {
case _: Infinite throw new IllegalArgumentException("illegal division of infinities")
// maybe questionable but pragmatic: Inf / 0 => Inf
case x Double.PositiveInfinity * (if ((this > Zero) ^ (other >= Zero)) -1 else 1)
}
def finite_? = false
def length: Long = throw new IllegalArgumentException("length not allowed on infinite Durations")
def unit: TimeUnit = throw new IllegalArgumentException("unit not allowed on infinite Durations")
def toNanos: Long = throw new IllegalArgumentException("toNanos not allowed on infinite Durations")
def toMicros: Long = throw new IllegalArgumentException("toMicros not allowed on infinite Durations")
def toMillis: Long = throw new IllegalArgumentException("toMillis not allowed on infinite Durations")
def toSeconds: Long = throw new IllegalArgumentException("toSeconds not allowed on infinite Durations")
def toMinutes: Long = throw new IllegalArgumentException("toMinutes not allowed on infinite Durations")
def toHours: Long = throw new IllegalArgumentException("toHours not allowed on infinite Durations")
def toDays: Long = throw new IllegalArgumentException("toDays not allowed on infinite Durations")
def toUnit(unit: TimeUnit): Double = throw new IllegalArgumentException("toUnit not allowed on infinite Durations")
def printHMS = toString
}
/**
* Infinite duration: greater than any other and not equal to any other,
* including itself.
*/
val Inf: Duration = new Duration with Infinite {
override def toString: String = "Duration.Inf"
def compare(other: Duration): Int = if (other eq this) 0 else 1
def unary_- : Duration = MinusInf
}
/**
* Infinite negative duration: lesser than any other and not equal to any other,
* including itself.
*/
val MinusInf: Duration = new Duration with Infinite {
override def toString = "Duration.MinusInf"
def compare(other: Duration): Int = if (other eq this) 0 else -1
def unary_- : Duration = Inf
}
// Java Factories
def create(length: Long, unit: TimeUnit): FiniteDuration = apply(length, unit)
def create(length: Double, unit: TimeUnit): FiniteDuration = apply(length, unit)
def create(length: Long, unit: String): FiniteDuration = apply(length, unit)
def parse(s: String): Duration = unapply(s).get
implicit object DurationIsOrdered extends Ordering[Duration] {
def compare(a: Duration, b: Duration): Int = a compare b
}
}
/**
* Utility for working with java.util.concurrent.TimeUnit durations.
*
* <p/>
* Examples of usage from Java:
* <pre>
* import akka.util.FiniteDuration;
* import java.util.concurrent.TimeUnit;
*
* Duration duration = new FiniteDuration(100, MILLISECONDS);
* Duration duration = new FiniteDuration(5, "seconds");
*
* duration.toNanos();
* </pre>
*
* <p/>
* Examples of usage from Scala:
* <pre>
* import akka.util.Duration
* import java.util.concurrent.TimeUnit
*
* val duration = Duration(100, MILLISECONDS)
* val duration = Duration(100, "millis")
*
* duration.toNanos
* duration < 1.second
* duration <= Duration.Inf
* </pre>
*
* <p/>
* Implicits are also provided for Int, Long and Double. Example usage:
* <pre>
* import akka.util.duration._
*
* val duration = 100 millis
* </pre>
*
* Extractors, parsing and arithmetic are also included:
* <pre>
* val d = Duration("1.2 µs")
* val Duration(length, unit) = 5 millis
* val d2 = d * 2.5
* val d3 = d2 + 1.millisecond
* </pre>
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
abstract class Duration extends Serializable with Ordered[Duration] {
def length: Long
def unit: TimeUnit
def toNanos: Long
def toMicros: Long
def toMillis: Long
def toSeconds: Long
def toMinutes: Long
def toHours: Long
def toDays: Long
def toUnit(unit: TimeUnit): Double
def printHMS: String
def +(other: Duration): Duration
def -(other: Duration): Duration
def *(factor: Double): Duration
def /(factor: Double): Duration
def /(other: Duration): Double
def unary_- : Duration
def finite_? : Boolean
def min(other: Duration): Duration = if (this < other) this else other
def max(other: Duration): Duration = if (this > other) this else other
def sleep(): Unit = Thread.sleep(toMillis)
def fromNow: Deadline = Deadline.now + this
// Java API
def lt(other: Duration): Boolean = this < other
def lteq(other: Duration): Boolean = this <= other
def gt(other: Duration): Boolean = this > other
def gteq(other: Duration): Boolean = this >= other
def plus(other: Duration): Duration = this + other
def minus(other: Duration): Duration = this - other
def mul(factor: Double): Duration = this * factor
def div(factor: Double): Duration = this / factor
def div(other: Duration): Double = this / other
def neg(): Duration = -this
def isFinite(): Boolean = finite_?
}
object FiniteDuration {
implicit object FiniteDurationIsOrdered extends Ordering[FiniteDuration] {
def compare(a: FiniteDuration, b: FiniteDuration) = a compare b
}
}
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
class FiniteDuration(val length: Long, val unit: TimeUnit) extends Duration {
import Duration._
require {
unit match {
/*
* sorted so that the first cases should be most-used ones, because enum
* is checked one after the other.
*/
case NANOSECONDS true
case MICROSECONDS length <= 9223372036854775L && length >= -9223372036854775L
case MILLISECONDS length <= 9223372036854L && length >= -9223372036854L
case SECONDS length <= 9223372036L && length >= -9223372036L
case MINUTES length <= 153722867L && length >= -153722867L
case HOURS length <= 2562047L && length >= -2562047L
case DAYS length <= 106751L && length >= -106751L
case _
val v = unit.convert(length, DAYS)
v <= 106751L && v >= -106751L
}
}
def this(length: Long, unit: String) = this(length, Duration.timeUnit(unit))
def toNanos = unit.toNanos(length)
def toMicros = unit.toMicros(length)
def toMillis = unit.toMillis(length)
def toSeconds = unit.toSeconds(length)
def toMinutes = unit.toMinutes(length)
def toHours = unit.toHours(length)
def toDays = unit.toDays(length)
def toUnit(u: TimeUnit) = toNanos.toDouble / NANOSECONDS.convert(1, u)
override def toString = this match {
case Duration(1, DAYS) "1 day"
case Duration(x, DAYS) x + " days"
case Duration(1, HOURS) "1 hour"
case Duration(x, HOURS) x + " hours"
case Duration(1, MINUTES) "1 minute"
case Duration(x, MINUTES) x + " minutes"
case Duration(1, SECONDS) "1 second"
case Duration(x, SECONDS) x + " seconds"
case Duration(1, MILLISECONDS) "1 millisecond"
case Duration(x, MILLISECONDS) x + " milliseconds"
case Duration(1, MICROSECONDS) "1 microsecond"
case Duration(x, MICROSECONDS) x + " microseconds"
case Duration(1, NANOSECONDS) "1 nanosecond"
case Duration(x, NANOSECONDS) x + " nanoseconds"
}
def printHMS = "%02d:%02d:%06.3f".format(toHours, toMinutes % 60, toMillis / 1000d % 60)
def compare(other: Duration) =
if (other.finite_?) {
val me = toNanos
val o = other.toNanos
if (me > o) 1 else if (me < o) -1 else 0
} else -other.compare(this)
private def add(a: Long, b: Long): Long = {
val c = a + b
// check if the signs of the top bit of both summands differ from the sum
if (((a ^ c) & (b ^ c)) < 0) throw new IllegalArgumentException("")
else c
}
def +(other: Duration): Duration = if (!other.finite_?) other else fromNanos(add(toNanos, other.toNanos))
def -(other: Duration): Duration = if (!other.finite_?) other else fromNanos(add(toNanos, -other.toNanos))
def *(factor: Double): FiniteDuration = fromNanos(toNanos.toDouble * factor)
def /(factor: Double): FiniteDuration = fromNanos(toNanos.toDouble / factor)
def /(other: Duration): Double = if (other.finite_?) toNanos.toDouble / other.toNanos else 0
def unary_- : FiniteDuration = Duration(-length, unit)
def finite_? : Boolean = true
override def equals(other: Any) =
(other.asInstanceOf[AnyRef] eq this) || other.isInstanceOf[FiniteDuration] &&
toNanos == other.asInstanceOf[FiniteDuration].toNanos
override def hashCode = {
val nanos = toNanos
(nanos ^ (nanos >> 32)).asInstanceOf[Int]
}
}
private[akka] trait DurationOps {
import duration.Classifier
protected def from(timeUnit: TimeUnit): FiniteDuration
def nanoseconds: FiniteDuration = from(NANOSECONDS)
def nanos: FiniteDuration = from(NANOSECONDS)
def nanosecond: FiniteDuration = from(NANOSECONDS)
def nano: FiniteDuration = from(NANOSECONDS)
def microseconds: FiniteDuration = from(MICROSECONDS)
def micros: FiniteDuration = from(MICROSECONDS)
def microsecond: FiniteDuration = from(MICROSECONDS)
def micro: FiniteDuration = from(MICROSECONDS)
def milliseconds: FiniteDuration = from(MILLISECONDS)
def millis: FiniteDuration = from(MILLISECONDS)
def millisecond: FiniteDuration = from(MILLISECONDS)
def milli: FiniteDuration = from(MILLISECONDS)
def seconds: FiniteDuration = from(SECONDS)
def second: FiniteDuration = from(SECONDS)
def minutes: FiniteDuration = from(MINUTES)
def minute: FiniteDuration = from(MINUTES)
def hours: FiniteDuration = from(HOURS)
def hour: FiniteDuration = from(HOURS)
def days: FiniteDuration = from(DAYS)
def day: FiniteDuration = from(DAYS)
def nanoseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(NANOSECONDS))
def nanos[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(NANOSECONDS))
def nanosecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(NANOSECONDS))
def nano[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(NANOSECONDS))
def microseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MICROSECONDS))
def micros[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MICROSECONDS))
def microsecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MICROSECONDS))
def micro[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MICROSECONDS))
def milliseconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MILLISECONDS))
def millis[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MILLISECONDS))
def millisecond[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MILLISECONDS))
def milli[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MILLISECONDS))
def seconds[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(SECONDS))
def second[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(SECONDS))
def minutes[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MINUTES))
def minute[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(MINUTES))
def hours[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(HOURS))
def hour[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(HOURS))
def days[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(DAYS))
def day[C, CC <: Classifier[C]](c: C)(implicit ev: CC): CC#R = ev.convert(from(DAYS))
}
class DurationInt(n: Int) extends DurationOps {
override protected def from(timeUnit: TimeUnit): FiniteDuration = Duration(n, timeUnit)
}
class DurationLong(n: Long) extends DurationOps {
override protected def from(timeUnit: TimeUnit): FiniteDuration = Duration(n, timeUnit)
}
class DurationDouble(d: Double) extends DurationOps {
override protected def from(timeUnit: TimeUnit): FiniteDuration = Duration(d, timeUnit)
}
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed //TODO add @SerialVersionUID(1L) when SI-4804 is fixed
case class Timeout(duration: Duration) { case class Timeout(duration: Duration) {

View file

@ -1,48 +0,0 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.util
import language.implicitConversions
import java.util.concurrent.TimeUnit
//FIXME Needs docs
package object duration {
trait Classifier[C] {
type R
def convert(d: FiniteDuration): R
}
object span
implicit object spanConvert extends Classifier[span.type] {
type R = FiniteDuration
def convert(d: FiniteDuration): FiniteDuration = d
}
object fromNow
implicit object fromNowConvert extends Classifier[fromNow.type] {
type R = Deadline
def convert(d: FiniteDuration): Deadline = Deadline.now + d
}
implicit def intToDurationInt(n: Int): DurationInt = new DurationInt(n)
implicit def longToDurationLong(n: Long): DurationLong = new DurationLong(n)
implicit def doubleToDurationDouble(d: Double): DurationDouble = new DurationDouble(d)
implicit def pairIntToDuration(p: (Int, TimeUnit)): FiniteDuration = Duration(p._1, p._2)
implicit def pairLongToDuration(p: (Long, TimeUnit)): FiniteDuration = Duration(p._1, p._2)
implicit def durationToPair(d: Duration): (Long, TimeUnit) = (d.length, d.unit)
/*
* avoid reflection based invocation by using non-duck type
*/
class IntMult(i: Int) { def *(d: Duration): Duration = d * i }
implicit def intMult(i: Int): IntMult = new IntMult(i)
class LongMult(l: Long) { def *(d: Duration): Duration = d * l }
implicit def longMult(l: Long): LongMult = new LongMult(l)
class DoubleMult(f: Double) { def *(d: Duration): Duration = d * f }
implicit def doubleMult(f: Double): DoubleMult = new DoubleMult(f)
}

View file

@ -3,8 +3,8 @@ package akka.agent
import language.postfixOps import language.postfixOps
import akka.dispatch.Await import akka.dispatch.Await
import akka.util.Duration import scala.concurrent.util.Duration
import akka.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._

View file

@ -4,12 +4,13 @@
package akka.camel package akka.camel
import internal._ import akka.camel.internal._
import akka.util.{ Timeout, Duration } import akka.util.Timeout
import akka.dispatch.Future import akka.dispatch.Future
import java.util.concurrent.TimeoutException import java.util.concurrent.TimeoutException
import akka.actor.{ ActorSystem, Props, ActorRef } import akka.actor.{ ActorSystem, Props, ActorRef }
import akka.pattern._ import akka.pattern._
import scala.concurrent.util.Duration
/** /**
* Activation trait that can be used to wait on activation or de-activation of Camel endpoints. * Activation trait that can be used to wait on activation or de-activation of Camel endpoints.

View file

@ -10,8 +10,8 @@ import internal.component.DurationTypeConverter
import org.apache.camel.model.{ RouteDefinition, ProcessorDefinition } import org.apache.camel.model.{ RouteDefinition, ProcessorDefinition }
import akka.actor._ import akka.actor._
import akka.util.Duration import scala.concurrent.util.Duration
import akka.util.duration._ import scala.concurrent.util.duration._
/** /**
* Mixed in by Actor implementations that consume message from Camel endpoints. * Mixed in by Actor implementations that consume message from Camel endpoints.

View file

@ -14,7 +14,7 @@ import akka.actor._
import collection.mutable import collection.mutable
import org.apache.camel.model.RouteDefinition import org.apache.camel.model.RouteDefinition
import org.apache.camel.CamelContext import org.apache.camel.CamelContext
import akka.util.Duration import scala.concurrent.util.Duration
/** /**
* For internal use only. * For internal use only.

View file

@ -1,12 +1,13 @@
package akka.camel.internal package akka.camel.internal
import akka.actor.ActorSystem import akka.actor.ActorSystem
import component.{ DurationTypeConverter, ActorComponent } import akka.camel.internal.component.{ DurationTypeConverter, ActorComponent }
import org.apache.camel.impl.DefaultCamelContext import org.apache.camel.impl.DefaultCamelContext
import scala.Predef._ import scala.Predef._
import akka.event.Logging import akka.event.Logging
import akka.camel.Camel import akka.camel.Camel
import akka.util.{ NonFatal, Duration } import akka.util.NonFatal
import scala.concurrent.util.Duration
import org.apache.camel.{ ProducerTemplate, CamelContext } import org.apache.camel.{ ProducerTemplate, CamelContext }
/** /**

View file

@ -15,10 +15,11 @@ import akka.actor._
import akka.pattern._ import akka.pattern._
import scala.reflect.BeanProperty import scala.reflect.BeanProperty
import akka.util.duration._ import scala.concurrent.util.duration._
import scala.concurrent.util.Duration
import java.util.concurrent.{ TimeoutException, CountDownLatch } import java.util.concurrent.{ TimeoutException, CountDownLatch }
import akka.camel.internal.CamelExchangeAdapter import akka.camel.internal.CamelExchangeAdapter
import akka.util.{ NonFatal, Duration, Timeout } import akka.util.{ NonFatal, Timeout }
import akka.camel.{ ActorNotRegisteredException, ConsumerConfig, Camel, Ack, FailureResult, CamelMessage } import akka.camel.{ ActorNotRegisteredException, ConsumerConfig, Camel, Ack, FailureResult, CamelMessage }
/** /**

View file

@ -7,7 +7,7 @@ package akka.camel;
import akka.actor.ActorRef; import akka.actor.ActorRef;
import akka.actor.ActorSystem; import akka.actor.ActorSystem;
import akka.actor.Props; import akka.actor.Props;
import akka.util.FiniteDuration; import scala.concurrent.util.FiniteDuration;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Test; import org.junit.Test;

View file

@ -6,7 +6,7 @@ package akka.camel;
import akka.actor.Status; import akka.actor.Status;
import akka.camel.javaapi.UntypedConsumerActor; import akka.camel.javaapi.UntypedConsumerActor;
import akka.util.Duration; import scala.concurrent.util.Duration;
import org.apache.camel.builder.Builder; import org.apache.camel.builder.Builder;
import org.apache.camel.model.ProcessorDefinition; import org.apache.camel.model.ProcessorDefinition;
import org.apache.camel.model.RouteDefinition; import org.apache.camel.model.RouteDefinition;

View file

@ -7,7 +7,7 @@ package akka.camel
import language.postfixOps import language.postfixOps
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import akka.util.duration._ import scala.concurrent.util.duration._
import org.apache.camel.ProducerTemplate import org.apache.camel.ProducerTemplate
import akka.actor._ import akka.actor._
import akka.util.Timeout import akka.util.Timeout

View file

@ -9,7 +9,7 @@ import language.existentials
import akka.actor._ import akka.actor._
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import akka.util.duration._ import scala.concurrent.util.duration._
import TestSupport._ import TestSupport._
import org.scalatest.WordSpec import org.scalatest.WordSpec
import org.apache.camel.model.RouteDefinition import org.apache.camel.model.RouteDefinition

View file

@ -15,7 +15,7 @@ import akka.actor.SupervisorStrategy.Stop
import org.scalatest.{ BeforeAndAfterEach, BeforeAndAfterAll, WordSpec } import org.scalatest.{ BeforeAndAfterEach, BeforeAndAfterAll, WordSpec }
import akka.actor._ import akka.actor._
import akka.pattern._ import akka.pattern._
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.util.Timeout import akka.util.Timeout
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import akka.testkit.TestLatch import akka.testkit.TestLatch

View file

@ -9,7 +9,7 @@ import language.postfixOps
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import org.scalatest.WordSpec import org.scalatest.WordSpec
import akka.camel.TestSupport.SharedCamelSystem import akka.camel.TestSupport.SharedCamelSystem
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.actor.{ ActorRef, Props } import akka.actor.{ ActorRef, Props }
class ProducerRegistryTest extends WordSpec with MustMatchers with SharedCamelSystem { class ProducerRegistryTest extends WordSpec with MustMatchers with SharedCamelSystem {

View file

@ -8,11 +8,11 @@ import language.postfixOps
import language.implicitConversions import language.implicitConversions
import akka.actor.{ Props, ActorSystem, Actor } import akka.actor.{ Props, ActorSystem, Actor }
import akka.util.duration._ import scala.concurrent.util.duration._
import java.util.concurrent.{ TimeoutException, ExecutionException, TimeUnit } import java.util.concurrent.{ TimeoutException, ExecutionException, TimeUnit }
import org.scalatest.{ BeforeAndAfterEach, BeforeAndAfterAll, Suite } import org.scalatest.{ BeforeAndAfterEach, BeforeAndAfterAll, Suite }
import org.scalatest.matchers.{ BePropertyMatcher, BePropertyMatchResult } import org.scalatest.matchers.{ BePropertyMatcher, BePropertyMatchResult }
import akka.util.{ FiniteDuration, Duration } import scala.concurrent.util.{ FiniteDuration, Duration }
private[camel] object TestSupport { private[camel] object TestSupport {

View file

@ -14,7 +14,7 @@ import akka.camel.TestSupport.SharedCamelSystem
import akka.actor.Props import akka.actor.Props
import akka.pattern._ import akka.pattern._
import akka.dispatch.Await import akka.dispatch.Await
import akka.util.duration._ import scala.concurrent.util.duration._
import org.scalatest._ import org.scalatest._
import matchers.MustMatchers import matchers.MustMatchers

View file

@ -3,10 +3,10 @@ package akka.camel.internal
import language.postfixOps import language.postfixOps
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import akka.util.duration._ import scala.concurrent.util.duration._
import org.scalatest.{ GivenWhenThen, BeforeAndAfterEach, BeforeAndAfterAll, WordSpec } import org.scalatest.{ GivenWhenThen, BeforeAndAfterEach, BeforeAndAfterAll, WordSpec }
import akka.actor.{ Props, ActorSystem } import akka.actor.{ Props, ActorSystem }
import akka.util.Duration import scala.concurrent.util.Duration
import akka.camel._ import akka.camel._
import akka.testkit.{ TimingTest, TestProbe, TestKit } import akka.testkit.{ TimingTest, TestProbe, TestKit }

View file

@ -7,7 +7,7 @@ package akka.camel.internal.component
import language.postfixOps import language.postfixOps
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.camel.TestSupport.SharedCamelSystem import akka.camel.TestSupport.SharedCamelSystem
import org.apache.camel.Component import org.apache.camel.Component
import org.scalatest.WordSpec import org.scalatest.WordSpec

View file

@ -11,8 +11,8 @@ import org.mockito.Matchers.{ eq ⇒ the, any }
import org.mockito.Mockito._ import org.mockito.Mockito._
import org.apache.camel.AsyncCallback import org.apache.camel.AsyncCallback
import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicBoolean
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.util.Duration import scala.concurrent.util.Duration
import akka.testkit.{ TestKit, TestProbe } import akka.testkit.{ TestKit, TestProbe }
import java.lang.String import java.lang.String
import akka.actor.{ ActorRef, Props, ActorSystem, Actor } import akka.actor.{ ActorRef, Props, ActorSystem, Actor }

View file

@ -7,8 +7,8 @@ package akka.camel.internal.component
import language.postfixOps import language.postfixOps
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.util.Duration import scala.concurrent.util.Duration
import org.scalatest.WordSpec import org.scalatest.WordSpec
import org.apache.camel.NoTypeConversionAvailableException import org.apache.camel.NoTypeConversionAvailableException

View file

@ -7,7 +7,7 @@ package akka.camelexamples
import language.postfixOps import language.postfixOps
import akka.camel._ import akka.camel._
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.actor.{ Actor, OneForOneStrategy } import akka.actor.{ Actor, OneForOneStrategy }
import akka.actor.SupervisorStrategy._ import akka.actor.SupervisorStrategy._

View file

@ -11,8 +11,8 @@ import scala.collection.immutable.Map
import scala.annotation.tailrec import scala.annotation.tailrec
import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.atomic.AtomicReference
import java.util.concurrent.TimeUnit.NANOSECONDS import java.util.concurrent.TimeUnit.NANOSECONDS
import akka.util.Duration import scala.concurrent.util.Duration
import akka.util.duration._ import scala.concurrent.util.duration._
object AccrualFailureDetector { object AccrualFailureDetector {
private def realClock: () Long = () NANOSECONDS.toMillis(System.nanoTime) private def realClock: () Long = () NANOSECONDS.toMillis(System.nanoTime)

View file

@ -4,18 +4,20 @@
package akka.cluster package akka.cluster
import language.implicitConversions
import akka.actor._ import akka.actor._
import akka.actor.Status._ import akka.actor.Status._
import akka.ConfigurationException import akka.ConfigurationException
import akka.dispatch.Await import akka.dispatch.{ Await, MonitorableThreadFactory }
import akka.dispatch.MonitorableThreadFactory
import akka.event.Logging import akka.event.Logging
import scala.concurrent.forkjoin.ThreadLocalRandom import scala.concurrent.forkjoin.ThreadLocalRandom
import akka.pattern._ import akka.pattern._
import akka.remote._ import akka.remote._
import akka.routing._ import akka.routing._
import akka.util._ import akka.util._
import akka.util.duration._ import scala.concurrent.util.duration._
import scala.concurrent.util.{ Duration, Deadline }
import akka.util.internal.HashedWheelTimer import akka.util.internal.HashedWheelTimer
import com.google.protobuf.ByteString import com.google.protobuf.ByteString
import java.io.Closeable import java.io.Closeable

View file

@ -4,7 +4,7 @@
package akka.cluster package akka.cluster
import com.typesafe.config.Config import com.typesafe.config.Config
import akka.util.Duration import scala.concurrent.util.Duration
import java.util.concurrent.TimeUnit.MILLISECONDS import java.util.concurrent.TimeUnit.MILLISECONDS
import akka.ConfigurationException import akka.ConfigurationException
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._

View file

@ -9,7 +9,7 @@ import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.atomic.AtomicLong
import akka.actor.Scheduler import akka.actor.Scheduler
import akka.util.Duration import scala.concurrent.util.Duration
/** /**
* INTERNAL API * INTERNAL API

View file

@ -6,7 +6,7 @@ package akka.cluster
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec import akka.remote.testkit.MultiNodeSpec
import akka.util.duration._ import scala.concurrent.util.duration._
import akka.testkit._ import akka.testkit._
object ClusterAccrualFailureDetectorMultiJvmSpec extends MultiNodeConfig { object ClusterAccrualFailureDetectorMultiJvmSpec extends MultiNodeConfig {

Some files were not shown because too many files have changed in this diff Show more