diff --git a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala index a9528a57aa..212ee9372d 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorSystem.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorSystem.scala @@ -676,8 +676,8 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config, def hasExtension(ext: ExtensionId[_ <: Extension]): Boolean = findExtension(ext) != null private def loadExtensions() { - import scala.collection.JavaConversions._ - settings.config.getStringList("akka.extensions") foreach { fqcn ⇒ + import scala.collection.JavaConverters.collectionAsScalaIterableConverter + settings.config.getStringList("akka.extensions").asScala foreach { fqcn ⇒ dynamicAccess.getObjectFor[AnyRef](fqcn) recoverWith { case _ ⇒ dynamicAccess.createInstanceFor[AnyRef](fqcn, Seq()) } match { case Success(p: ExtensionIdProvider) ⇒ registerExtension(p.lookup()) case Success(p: ExtensionId[_]) ⇒ registerExtension(p) diff --git a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala index fbee491664..d72389ae5e 100644 --- a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala @@ -7,7 +7,6 @@ import language.implicitConversions import java.util.concurrent.TimeUnit import scala.collection.mutable.ArrayBuffer -import scala.collection.JavaConversions._ import java.lang.{ Iterable ⇒ JIterable } import scala.concurrent.duration.Duration /** @@ -195,7 +194,10 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits { * Decider builder which just checks whether one of * the given Throwables matches the cause and restarts, otherwise escalates. */ - def makeDecider(trapExit: JIterable[Class[_ <: Throwable]]): Decider = makeDecider(trapExit.toSeq) + def makeDecider(trapExit: JIterable[Class[_ <: Throwable]]): Decider = { + import scala.collection.JavaConverters.iterableAsScalaIterableConverter + makeDecider(trapExit.asScala.toSeq) + } /** * Decider builder for Iterables of cause-directive pairs, e.g. a map obtained diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index 113215cd23..59c0afe5aa 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -68,7 +68,7 @@ object ExecutionContexts { * Futures is the Java API for Futures and Promises */ object Futures { - + import scala.collection.JavaConverters.iterableAsScalaIterableConverter /** * Java API, equivalent to Future.apply */ @@ -95,7 +95,7 @@ object Futures { */ def find[T <: AnyRef](futures: JIterable[Future[T]], predicate: JFunc[T, java.lang.Boolean], executor: ExecutionContext): Future[JOption[T]] = { implicit val ec = executor - Future.find[T]((scala.collection.JavaConversions.iterableAsScalaIterable(futures)))(predicate.apply(_))(executor).map(JOption.fromScalaOption(_)) + Future.find[T](futures.asScala)(predicate.apply(_))(executor).map(JOption.fromScalaOption(_)) } /** @@ -103,7 +103,7 @@ object Futures { * Returns a Future to the result of the first future in the list that is completed */ def firstCompletedOf[T <: AnyRef](futures: JIterable[Future[T]], executor: ExecutionContext): Future[T] = - Future.firstCompletedOf(scala.collection.JavaConversions.iterableAsScalaIterable(futures))(executor) + Future.firstCompletedOf(futures.asScala)(executor) /** * Java API @@ -113,14 +113,14 @@ object Futures { * or the result of the fold. */ def fold[T <: AnyRef, R <: AnyRef](zero: R, futures: JIterable[Future[T]], fun: akka.japi.Function2[R, T, R], executor: ExecutionContext): Future[R] = - Future.fold(scala.collection.JavaConversions.iterableAsScalaIterable(futures))(zero)(fun.apply)(executor) + Future.fold(futures.asScala)(zero)(fun.apply)(executor) /** * Java API. * Reduces the results of the supplied futures and binary function. */ def reduce[T <: AnyRef, R >: T](futures: JIterable[Future[T]], fun: akka.japi.Function2[R, T, R], executor: ExecutionContext): Future[R] = - Future.reduce[T, R](scala.collection.JavaConversions.iterableAsScalaIterable(futures))(fun.apply)(executor) + Future.reduce[T, R](futures.asScala)(fun.apply)(executor) /** * Java API. @@ -129,9 +129,7 @@ object Futures { */ def sequence[A](in: JIterable[Future[A]], executor: ExecutionContext): Future[JIterable[A]] = { implicit val d = executor - scala.collection.JavaConversions.iterableAsScalaIterable(in).foldLeft(Future(new JLinkedList[A]())) { (fr, fa) ⇒ - for (r ← fr; a ← fa) yield { r add a; r } - } + in.asScala.foldLeft(Future(new JLinkedList[A]())) { (fr, fa) ⇒ for (r ← fr; a ← fa) yield { r add a; r } } } /** @@ -142,7 +140,7 @@ object Futures { */ def traverse[A, B](in: JIterable[A], fn: JFunc[A, Future[B]], executor: ExecutionContext): Future[JIterable[B]] = { implicit val d = executor - scala.collection.JavaConversions.iterableAsScalaIterable(in).foldLeft(Future(new JLinkedList[B]())) { (fr, a) ⇒ + in.asScala.foldLeft(Future(new JLinkedList[B]())) { (fr, a) ⇒ val fb = fn(a) for (r ← fr; b ← fb) yield { r add b; r } } diff --git a/akka-actor/src/main/scala/akka/japi/JavaAPI.scala b/akka-actor/src/main/scala/akka/japi/JavaAPI.scala index 642600f2bd..ee2a688345 100644 --- a/akka-actor/src/main/scala/akka/japi/JavaAPI.scala +++ b/akka-actor/src/main/scala/akka/japi/JavaAPI.scala @@ -9,6 +9,7 @@ import scala.Some import scala.reflect.ClassTag import scala.util.control.NoStackTrace import scala.runtime.AbstractPartialFunction +import java.util.Collections.{ emptyList, singletonList } /** * A Function interface. Used to create first-class-functions is Java. @@ -114,13 +115,11 @@ abstract class JavaPartialFunction[A, B] extends AbstractPartialFunction[A, B] { * Java API */ sealed abstract class Option[A] extends java.lang.Iterable[A] { - import scala.collection.JavaConversions._ - def get: A def isEmpty: Boolean def isDefined: Boolean = !isEmpty def asScala: scala.Option[A] - def iterator: java.util.Iterator[A] = if (isEmpty) Iterator.empty else Iterator.single(get) + def iterator: java.util.Iterator[A] = if (isEmpty) emptyList[A].iterator else singletonList(get).iterator } object Option { diff --git a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala index cdfd040ace..71a36be2b7 100644 --- a/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala +++ b/akka-actor/src/main/scala/akka/routing/ConsistentHashingRouter.scala @@ -3,7 +3,7 @@ */ package akka.routing -import scala.collection.JavaConversions.iterableAsScalaIterable +import scala.collection.JavaConverters.iterableAsScalaIterableConverter import scala.util.control.NonFatal import akka.actor.ActorRef import akka.actor.SupervisorStrategy @@ -163,7 +163,7 @@ case class ConsistentHashingRouter( * @param routeePaths string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ - def this(routeePaths: java.lang.Iterable[String]) = this(routees = iterableAsScalaIterable(routeePaths)) + def this(routeePaths: java.lang.Iterable[String]) = this(routees = routeePaths.asScala) /** * Constructor that sets the resizer to be used. diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index ac4f55bd7c..9ccf43fb58 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -10,7 +10,7 @@ import scala.concurrent.duration._ import akka.ConfigurationException import akka.pattern.pipe import com.typesafe.config.Config -import scala.collection.JavaConversions.iterableAsScalaIterable +import scala.collection.JavaConverters.iterableAsScalaIterableConverter import java.util.concurrent.atomic.{ AtomicLong, AtomicBoolean } import java.util.concurrent.TimeUnit import scala.concurrent.forkjoin.ThreadLocalRandom @@ -335,7 +335,7 @@ abstract class CustomRouterConfig extends RouterConfig { val customRoute = createCustomRoute(routeeProvider) { - case (sender, message) ⇒ customRoute.destinationsFor(sender, message) + case (sender, message) ⇒ customRoute.destinationsFor(sender, message).asScala } } @@ -564,7 +564,7 @@ case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] = * @param routeePaths string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ - def this(routeePaths: java.lang.Iterable[String]) = this(routees = iterableAsScalaIterable(routeePaths)) + def this(routeePaths: java.lang.Iterable[String]) = this(routees = routeePaths.asScala) /** * Constructor that sets the resizer to be used. @@ -701,7 +701,7 @@ case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, * @param routeePaths string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ - def this(routeePaths: java.lang.Iterable[String]) = this(routees = iterableAsScalaIterable(routeePaths)) + def this(routeePaths: java.lang.Iterable[String]) = this(routees = routeePaths.asScala) /** * Constructor that sets the resizer to be used. @@ -845,7 +845,7 @@ case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: Iterable[Strin * @param routeePaths string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ - def this(routeePaths: java.lang.Iterable[String]) = this(routees = iterableAsScalaIterable(routeePaths)) + def this(routeePaths: java.lang.Iterable[String]) = this(routees = routeePaths.asScala) /** * Constructor that sets the resizer to be used. @@ -1064,7 +1064,7 @@ case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = N * @param routeePaths string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ - def this(routeePaths: java.lang.Iterable[String]) = this(routees = iterableAsScalaIterable(routeePaths)) + def this(routeePaths: java.lang.Iterable[String]) = this(routees = routeePaths.asScala) /** * Constructor that sets the resizer to be used. @@ -1196,8 +1196,7 @@ case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: It * @param routeePaths string representation of the actor paths of the routees that will be looked up * using `actorFor` in [[akka.actor.ActorRefProvider]] */ - def this(routeePaths: java.lang.Iterable[String], w: FiniteDuration) = - this(routees = iterableAsScalaIterable(routeePaths), within = w) + def this(routeePaths: java.lang.Iterable[String], w: FiniteDuration) = this(routees = routeePaths.asScala, within = w) /** * Constructor that sets the resizer to be used. diff --git a/akka-actor/src/main/scala/akka/util/Index.scala b/akka-actor/src/main/scala/akka/util/Index.scala index 3289ed8f13..ccaab901c3 100644 --- a/akka-actor/src/main/scala/akka/util/Index.scala +++ b/akka-actor/src/main/scala/akka/util/Index.scala @@ -7,6 +7,7 @@ import annotation.tailrec import java.util.concurrent.{ ConcurrentSkipListSet, ConcurrentHashMap } import java.util.{ Comparator, Set ⇒ JSet } +import scala.collection.JavaConverters.{ asScalaIteratorConverter, collectionAsScalaIterableConverter } import scala.collection.mutable /** @@ -71,12 +72,11 @@ class Index[K, V](val mapSize: Int, val valueComparator: Comparator[V]) { * @return Some(value) for the first matching value where the supplied function returns true for the given key, * if no matches it returns None */ - def findValue(key: K)(f: (V) ⇒ Boolean): Option[V] = { - import scala.collection.JavaConversions._ - val set = container get key - if (set ne null) set.iterator.find(f) - else None - } + def findValue(key: K)(f: (V) ⇒ Boolean): Option[V] = + container get key match { + case null ⇒ None + case set ⇒ set.iterator.asScala.find(f) + } /** * Returns an Iterator of V containing the values for the supplied key, or an empty iterator if the key doesn't exist @@ -84,27 +84,24 @@ class Index[K, V](val mapSize: Int, val valueComparator: Comparator[V]) { def valueIterator(key: K): scala.Iterator[V] = { container.get(key) match { case null ⇒ Iterator.empty - case some ⇒ scala.collection.JavaConversions.asScalaIterator(some.iterator()) + case some ⇒ some.iterator.asScala } } /** * Applies the supplied function to all keys and their values */ - def foreach(fun: (K, V) ⇒ Unit): Unit = { - import scala.collection.JavaConversions._ - container.entrySet foreach { e ⇒ e.getValue.foreach(fun(e.getKey, _)) } - } + def foreach(fun: (K, V) ⇒ Unit): Unit = + container.entrySet.iterator.asScala foreach { e ⇒ e.getValue.iterator.asScala.foreach(fun(e.getKey, _)) } /** * Returns the union of all value sets. */ def values: Set[V] = { - import scala.collection.JavaConversions._ val builder = mutable.Set.empty[V] for { - entry ← container.entrySet - v ← entry.getValue + values ← container.values.iterator.asScala + v ← values.iterator.asScala } builder += v builder.toSet } @@ -112,7 +109,7 @@ class Index[K, V](val mapSize: Int, val valueComparator: Comparator[V]) { /** * Returns the key set. */ - def keys: Iterable[K] = scala.collection.JavaConversions.collectionAsScalaIterable(container.keySet) + def keys: Iterable[K] = container.keySet.asScala /** * Disassociates the value of type V from the key of type K diff --git a/akka-camel/src/main/scala/akka/camel/CamelMessage.scala b/akka-camel/src/main/scala/akka/camel/CamelMessage.scala index 8b0dbef50a..70fd61bd2a 100644 --- a/akka-camel/src/main/scala/akka/camel/CamelMessage.scala +++ b/akka-camel/src/main/scala/akka/camel/CamelMessage.scala @@ -5,21 +5,18 @@ package akka.camel import java.util.{ Map ⇒ JMap, Set ⇒ JSet } - -import scala.collection.JavaConversions._ - import org.apache.camel.{ CamelContext, Message ⇒ JCamelMessage } import akka.AkkaException import scala.reflect.ClassTag +import scala.util.Try +import scala.collection.JavaConversions._ import akka.dispatch.Mapper -import util.{ Success, Failure, Try } /** * An immutable representation of a Camel message. * @author Martin Krasser */ case class CamelMessage(body: Any, headers: Map[String, Any]) { - def this(body: Any, headers: JMap[String, Any]) = this(body, headers.toMap) //for Java override def toString: String = "CamelMessage(%s, %s)" format (body, headers) @@ -76,8 +73,7 @@ case class CamelMessage(body: Any, headers: Map[String, Any]) { *
* Java API
*/
- def getHeaderAs[T](name: String, clazz: Class[T], camelContext: CamelContext): T =
- headerAs[T](name)(ClassTag(clazz), camelContext).get
+ def getHeaderAs[T](name: String, clazz: Class[T], camelContext: CamelContext): T = headerAs[T](name)(ClassTag(clazz), camelContext).get
/**
* Returns a new CamelMessage with a transformed body using a transformer function.
diff --git a/akka-camel/src/main/scala/akka/camel/internal/CamelExchangeAdapter.scala b/akka-camel/src/main/scala/akka/camel/internal/CamelExchangeAdapter.scala
index 5750856b37..1d16c3003e 100644
--- a/akka-camel/src/main/scala/akka/camel/internal/CamelExchangeAdapter.scala
+++ b/akka-camel/src/main/scala/akka/camel/internal/CamelExchangeAdapter.scala
@@ -1,9 +1,6 @@
package akka.camel.internal
-import scala.collection.JavaConversions._
-
import org.apache.camel.util.ExchangeHelper
-
import org.apache.camel.{ Exchange, Message ⇒ JCamelMessage }
import akka.camel.{ FailureResult, AkkaCamelException, CamelMessage }
@@ -83,8 +80,10 @@ private[camel] class CamelExchangeAdapter(val exchange: Exchange) {
*
* @see AkkaCamelException
*/
- def toAkkaCamelException(headers: Map[String, Any]): AkkaCamelException =
+ def toAkkaCamelException(headers: Map[String, Any]): AkkaCamelException = {
+ import scala.collection.JavaConversions._
new AkkaCamelException(exchange.getException, headers ++ response.getHeaders)
+ }
/**
* Creates an immutable Failure object from the adapted Exchange so it can be used internally between Actors.
@@ -101,7 +100,10 @@ private[camel] class CamelExchangeAdapter(val exchange: Exchange) {
*
* @see Failure
*/
- def toFailureResult(headers: Map[String, Any]): FailureResult = FailureResult(exchange.getException, headers ++ response.getHeaders)
+ def toFailureResult(headers: Map[String, Any]): FailureResult = {
+ import scala.collection.JavaConversions._
+ FailureResult(exchange.getException, headers ++ response.getHeaders)
+ }
/**
* Creates an immutable CamelMessage object from Exchange.getIn so it can be used with Actors.
diff --git a/akka-transactor/src/main/scala/akka/transactor/UntypedTransactor.scala b/akka-transactor/src/main/scala/akka/transactor/UntypedTransactor.scala
index 98899e0a13..69fcac66ad 100644
--- a/akka-transactor/src/main/scala/akka/transactor/UntypedTransactor.scala
+++ b/akka-transactor/src/main/scala/akka/transactor/UntypedTransactor.scala
@@ -5,13 +5,15 @@
package akka.transactor
import akka.actor.{ UntypedActor, ActorRef }
-import scala.collection.JavaConversions._
import java.util.{ Set ⇒ JSet }
+import java.util.Collections.{ emptySet, singleton ⇒ singletonSet }
/**
* An UntypedActor version of transactor for using from Java.
*/
abstract class UntypedTransactor extends UntypedActor {
+ import scala.collection.JavaConverters.asScalaSetConverter
+
private val settings = TransactorExtension(context.system)
/**
@@ -21,8 +23,7 @@ abstract class UntypedTransactor extends UntypedActor {
final def onReceive(message: Any) {
message match {
case coordinated @ Coordinated(message) ⇒ {
- val others = coordinate(message)
- for (sendTo ← others) {
+ for (sendTo ← coordinate(message).asScala) {
sendTo.actor ! coordinated(sendTo.message.getOrElse(message))
}
before(message)
@@ -49,19 +50,19 @@ abstract class UntypedTransactor extends UntypedActor {
/**
* Empty set of transactors to send to.
*/
- def nobody: JSet[SendTo] = Set[SendTo]()
+ def nobody: JSet[SendTo] = emptySet()
/**
* For including one other actor in this coordinated transaction and sending
* them the same message as received. Use as the result in `coordinated`.
*/
- def include(actor: ActorRef): JSet[SendTo] = Set(SendTo(actor))
+ def include(actor: ActorRef): JSet[SendTo] = singletonSet(SendTo(actor))
/**
* For including one other actor in this coordinated transaction and specifying the
* message to send. Use as the result in `coordinated`.
*/
- def include(actor: ActorRef, message: Any): JSet[SendTo] = Set(SendTo(actor, Some(message)))
+ def include(actor: ActorRef, message: Any): JSet[SendTo] = singletonSet(SendTo(actor, Some(message)))
/**
* For including another actor in this coordinated transaction and sending