Various scala-2.13.0-M5 fixes

fix akka-actor-tests compile errors

some tests still fail though

Fix test failures in akka-actor-test

Manually work arround missing implicit Factory[Nothing, Seq[Nothing]]

see https://github.com/scala/scala-collection-compat/issues/137

akka-remote scalafix changes

Fix shutdownAll compile error

test:akka-remote scalafix changes

akka-multi-node-testkit scalafix

Fix akka-remote-tests multi-jvm compile errors

akka-stream-tests/test:scalafix

Fix test:akka-stream-tests

Crude implementation of ByteString.map

scalafix akka-actor-typed, akka-actor-typed-tests

akka-actor-typed-tests compile and succeed

scalafix akka-camel

scalafix akka-cluster

akka-cluster compile & test

scalafix akka-cluster-metrics

Fix akka-cluster-metrics

scalafix akka-cluster-tools

akka-cluster-tools compile and test

scalafix akka-distributed-data

akka-distributed-data fixes

scalafix akka-persistence

scalafix akka-cluster-sharding

fix akka-cluster-sharding

scalafix akka-contrib

Fix akka-cluster-sharding-typed test

scalafix akka-docs

Use scala-stm 0.9 (released for M5)

akka-docs

Remove dependency on collections-compat

Cherry-pick the relevant constructs to our own
private utils

Shorten 'scala.collections.immutable' by importing it

Duplicate 'immutable' imports

Use 'foreach' on futures

Replace MapLike with regular Map

Internal API markers

Simplify ccompat by moving PackageShared into object

Since we don't currently need to differentiate between 2.11 and

Avoid relying on 'union' (and ++) being left-biased

Fix akka-actor/doc by removing -Ywarn-unused

Make more things more private

Copyright headers

Use 'unsorted' to go from SortedSet to Set

Duplicate import

Use onComplete rather than failed.foreach

Clarify why we partly duplicate scala-collection-compat
This commit is contained in:
Arnout Engelen 2018-11-22 16:18:10 +01:00 committed by Arnout Engelen
parent 3bff646218
commit d274e039f9
No known key found for this signature in database
GPG key ID: BB8C0F854A1E2105
141 changed files with 596 additions and 468 deletions

View file

@ -10,6 +10,7 @@ import akka.actor.ActorSystem;
import akka.japi.*; import akka.japi.*;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;
import scala.Function1;
import scala.concurrent.Await; import scala.concurrent.Await;
import scala.concurrent.Future; import scala.concurrent.Future;
import scala.concurrent.Promise; import scala.concurrent.Promise;
@ -26,6 +27,7 @@ import java.util.concurrent.TimeUnit;
import static akka.japi.Util.classTag; import static akka.japi.Util.classTag;
import akka.testkit.AkkaSpec; import akka.testkit.AkkaSpec;
import scala.util.Try;
public class JavaFutureTests extends JUnitSuite { public class JavaFutureTests extends JUnitSuite {
@ -59,9 +61,9 @@ public class JavaFutureTests extends JUnitSuite {
final CountDownLatch latch = new CountDownLatch(1); final CountDownLatch latch = new CountDownLatch(1);
Promise<String> cf = Futures.promise(); Promise<String> cf = Futures.promise();
Future<String> f = cf.future(); Future<String> f = cf.future();
f.onSuccess(new OnSuccess<String>() { f.onComplete(new OnComplete<String>() {
public void onSuccess(String result) { public void onComplete(Throwable t, String r) {
if (result.equals("foo")) if ("foo".equals(r))
latch.countDown(); latch.countDown();
} }
}, system.dispatcher()); }, system.dispatcher());
@ -76,8 +78,9 @@ public class JavaFutureTests extends JUnitSuite {
final CountDownLatch latch = new CountDownLatch(1); final CountDownLatch latch = new CountDownLatch(1);
Promise<String> cf = Futures.promise(); Promise<String> cf = Futures.promise();
Future<String> f = cf.future(); Future<String> f = cf.future();
f.onFailure(new OnFailure() { f.onComplete(new OnComplete<String>() {
public void onFailure(Throwable t) { public void onComplete(Throwable t, String r) {
// 'null instanceof ...' is always false
if (t instanceof NullPointerException) if (t instanceof NullPointerException)
latch.countDown(); latch.countDown();
} }

View file

@ -8,7 +8,7 @@ import language.postfixOps
import java.io.Closeable import java.io.Closeable
import java.util.concurrent._ import java.util.concurrent._
import atomic.{ AtomicReference, AtomicInteger } import atomic.{ AtomicReference, AtomicInteger }
import scala.concurrent.{ future, Await, ExecutionContext } import scala.concurrent.{ Future, Await, ExecutionContext }
import scala.concurrent.duration._ import scala.concurrent.duration._
import java.util.concurrent.ThreadLocalRandom import java.util.concurrent.ThreadLocalRandom
import scala.util.Try import scala.util.Try
@ -378,7 +378,7 @@ class LightArrayRevolverSchedulerSpec extends AkkaSpec(SchedulerSpec.testConfRev
import driver._ import driver._
import system.dispatcher import system.dispatcher
val counter = new AtomicInteger val counter = new AtomicInteger
val terminated = future { val terminated = Future {
var rounds = 0 var rounds = 0
while (Try(sched.scheduleOnce(Duration.Zero)(())(localEC)).isSuccess) { while (Try(sched.scheduleOnce(Duration.Zero)(())(localEC)).isSuccess) {
Thread.sleep(1) Thread.sleep(1)
@ -511,7 +511,7 @@ class LightArrayRevolverSchedulerSpec extends AkkaSpec(SchedulerSpec.testConfRev
withScheduler() { (sched, driver) withScheduler() { (sched, driver)
import system.dispatcher import system.dispatcher
val counter = new AtomicInteger val counter = new AtomicInteger
future { Thread.sleep(5); driver.close(); sched.close() } Future { Thread.sleep(5); driver.close(); sched.close() }
val headroom = 200 val headroom = 200
var overrun = headroom var overrun = headroom
val cap = 1000000 val cap = 1000000

View file

@ -6,7 +6,8 @@ package akka.dispatch
import java.util.concurrent.{ ExecutorService, Executor, Executors } import java.util.concurrent.{ ExecutorService, Executor, Executors }
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import scala.concurrent.{ BatchingExecutor => _, Batchable => _, _ } import scala.concurrent.{ ExecutionContext, ExecutionContextExecutor, ExecutionContextExecutorService }
import scala.concurrent.{ Await, blocking, Promise, Future }
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.testkit.{ TestLatch, AkkaSpec, DefaultTimeout } import akka.testkit.{ TestLatch, AkkaSpec, DefaultTimeout }
import akka.util.SerializedSuspendableExecutionContext import akka.util.SerializedSuspendableExecutionContext

View file

@ -5,7 +5,7 @@
package akka.io package akka.io
import akka.testkit.{ TestProbe, AkkaSpec } import akka.testkit.{ TestProbe, AkkaSpec }
import akka.testkit.SocketUtil._ import akka.testkit.SocketUtil.temporaryServerAddresses
import Tcp._ import Tcp._
class CapacityLimitSpec extends AkkaSpec(""" class CapacityLimitSpec extends AkkaSpec("""

View file

@ -24,7 +24,7 @@ import akka.io.Inet.SocketOption
import akka.actor._ import akka.actor._
import akka.testkit.{ AkkaSpec, EventFilter, SocketUtil, TestActorRef, TestProbe } import akka.testkit.{ AkkaSpec, EventFilter, SocketUtil, TestActorRef, TestProbe }
import akka.util.{ ByteString, Helpers } import akka.util.{ ByteString, Helpers }
import akka.testkit.SocketUtil._ import akka.testkit.SocketUtil.temporaryServerAddress
import java.util.Random import java.util.Random
import java.net.SocketTimeoutException import java.net.SocketTimeoutException
import java.nio.file.Files import java.nio.file.Files

View file

@ -9,11 +9,10 @@ import scala.collection.immutable
import akka.testkit.{ AkkaSpec, TestProbe } import akka.testkit.{ AkkaSpec, TestProbe }
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.io.Inet.SocketOption import akka.io.Inet.SocketOption
import akka.testkit.SocketUtil._ import akka.testkit.SocketUtil.temporaryServerAddress
import Tcp._ import Tcp._
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.dispatch.ExecutionContexts import akka.dispatch.ExecutionContexts
import scala.collection.immutable
trait TcpIntegrationSpecSupport { _: AkkaSpec trait TcpIntegrationSpecSupport { _: AkkaSpec

View file

@ -8,7 +8,7 @@ import java.net.InetSocketAddress
import akka.testkit.{ TestProbe, ImplicitSender, AkkaSpec } import akka.testkit.{ TestProbe, ImplicitSender, AkkaSpec }
import akka.util.ByteString import akka.util.ByteString
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.testkit.SocketUtil._ import akka.testkit.SocketUtil.temporaryServerAddresses
class UdpConnectedIntegrationSpec extends AkkaSpec(""" class UdpConnectedIntegrationSpec extends AkkaSpec("""
akka.loglevel = INFO akka.loglevel = INFO

View file

@ -11,7 +11,7 @@ import akka.util.ByteString
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.io.Udp._ import akka.io.Udp._
import akka.io.Inet._ import akka.io.Inet._
import akka.testkit.SocketUtil._ import akka.testkit.SocketUtil.temporaryServerAddresses
import java.net.DatagramSocket import java.net.DatagramSocket
class UdpIntegrationSpec extends AkkaSpec(""" class UdpIntegrationSpec extends AkkaSpec("""

View file

@ -53,7 +53,7 @@ class RoundRobinSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
val doneLatch = new TestLatch(connectionCount) val doneLatch = new TestLatch(connectionCount)
val counter = new AtomicInteger val counter = new AtomicInteger
var replies = Map.empty[Int, Int].withDefaultValue(0) var replies: Map[Int, Int] = Map.empty.withDefaultValue(0)
val actor = system.actorOf(RoundRobinPool(connectionCount).props(routeeProps = Props(new Actor { val actor = system.actorOf(RoundRobinPool(connectionCount).props(routeeProps = Props(new Actor {
lazy val id = counter.getAndIncrement() lazy val id = counter.getAndIncrement()
@ -65,7 +65,7 @@ class RoundRobinSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
for (_ 1 to iterationCount; _ 1 to connectionCount) { for (_ 1 to iterationCount; _ 1 to connectionCount) {
val id = Await.result((actor ? "hit").mapTo[Int], timeout.duration) val id = Await.result((actor ? "hit").mapTo[Int], timeout.duration)
replies = replies + (id (replies(id) + 1)) replies += (id (replies(id) + 1))
} }
counter.get should ===(connectionCount) counter.get should ===(connectionCount)
@ -123,7 +123,7 @@ class RoundRobinSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
val iterationCount = 10 val iterationCount = 10
val doneLatch = new TestLatch(connectionCount) val doneLatch = new TestLatch(connectionCount)
var replies = Map.empty[String, Int].withDefaultValue(0) var replies: Map[String, Int] = Map.empty.withDefaultValue(0)
val paths = (1 to connectionCount) map { n val paths = (1 to connectionCount) map { n
val ref = system.actorOf(Props(new Actor { val ref = system.actorOf(Props(new Actor {
@ -139,7 +139,7 @@ class RoundRobinSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
for (_ 1 to iterationCount; _ 1 to connectionCount) { for (_ 1 to iterationCount; _ 1 to connectionCount) {
val id = Await.result((actor ? "hit").mapTo[String], timeout.duration) val id = Await.result((actor ? "hit").mapTo[String], timeout.duration)
replies = replies + (id (replies(id) + 1)) replies += (id (replies(id) + 1))
} }
actor ! akka.routing.Broadcast("end") actor ! akka.routing.Broadcast("end")
@ -154,7 +154,7 @@ class RoundRobinSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
val connectionCount = 10 val connectionCount = 10
val iterationCount = 10 val iterationCount = 10
var replies = Map.empty[String, Int].withDefaultValue(0) var replies: Map[String, Int] = Map.empty.withDefaultValue(0)
val actor = system.actorOf(Props(new Actor { val actor = system.actorOf(Props(new Actor {
var n = 0 var n = 0
@ -185,7 +185,7 @@ class RoundRobinSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
for (_ 1 to iterationCount; _ 1 to connectionCount) { for (_ 1 to iterationCount; _ 1 to connectionCount) {
val id = Await.result((actor ? "hit").mapTo[String], timeout.duration) val id = Await.result((actor ? "hit").mapTo[String], timeout.duration)
replies = replies + (id (replies(id) + 1)) replies += (id (replies(id) + 1))
} }
watch(actor) watch(actor)

View file

@ -136,7 +136,7 @@ class BoundedBlockingQueueSpec
} }
mustBlockFor(100 milliseconds, f) mustBlockFor(100 milliseconds, f)
events should containInSequence(offer("a"), awaitNotFull, signalNotFull, getSize, awaitNotFull) events.toList should containInSequence(offer("a"), awaitNotFull, signalNotFull, getSize, awaitNotFull)
events shouldNot contain(offer("b")) events shouldNot contain(offer("b"))
} }
} }
@ -196,7 +196,7 @@ class BoundedBlockingQueueSpec
// `f` should still block since the queue is still empty // `f` should still block since the queue is still empty
mustBlockFor(100 milliseconds, f) mustBlockFor(100 milliseconds, f)
events should containInSequence(getSize, awaitNotEmpty, signalNotEmpty, getSize, awaitNotEmpty) events.toList should containInSequence(getSize, awaitNotEmpty, signalNotEmpty, getSize, awaitNotEmpty)
events shouldNot contain(poll) events shouldNot contain(poll)
} }
} }
@ -297,7 +297,7 @@ class BoundedBlockingQueueSpec
// `f` should still block since the queue is still empty // `f` should still block since the queue is still empty
mustBlockFor(100 milliseconds, f) mustBlockFor(100 milliseconds, f)
events should containInSequence(getSize, awaitNotFull, signalNotFull, getSize, awaitNotFull) events.toList should containInSequence(getSize, awaitNotFull, signalNotFull, getSize, awaitNotFull)
events shouldNot contain(offer("World")) events shouldNot contain(offer("World"))
} }
} }
@ -706,12 +706,12 @@ trait QueueSetupHelper {
import akka.util.QueueTestEvents._ import akka.util.QueueTestEvents._
case class TestContext(queue: BoundedBlockingQueue[String], events: mutable.MutableList[QueueEvent], notEmpty: TestCondition, notFull: TestCondition, lock: ReentrantLock, backingQueue: util.Queue[String]) case class TestContext(queue: BoundedBlockingQueue[String], events: mutable.Buffer[QueueEvent], notEmpty: TestCondition, notFull: TestCondition, lock: ReentrantLock, backingQueue: util.Queue[String])
/** /**
* Backing queue that records all poll and offer calls in `events` * Backing queue that records all poll and offer calls in `events`
*/ */
class TestBackingQueue(events: mutable.MutableList[QueueEvent]) class TestBackingQueue(events: mutable.Buffer[QueueEvent])
extends util.LinkedList[String] { extends util.LinkedList[String] {
override def poll(): String = { override def poll(): String = {
@ -733,7 +733,7 @@ trait QueueSetupHelper {
/** /**
* Reentrant lock condition that records when the condition is signaled or `await`ed. * Reentrant lock condition that records when the condition is signaled or `await`ed.
*/ */
class TestCondition(events: mutable.MutableList[QueueEvent], condition: Condition, signalEvent: QueueEvent, awaitEvent: QueueEvent) class TestCondition(events: mutable.Buffer[QueueEvent], condition: Condition, signalEvent: QueueEvent, awaitEvent: QueueEvent)
extends Condition { extends Condition {
case class Manual(waitTime: Long = 0, waitingThread: Option[Thread] = None) case class Manual(waitTime: Long = 0, waitingThread: Option[Thread] = None)
@ -798,7 +798,7 @@ trait QueueSetupHelper {
} }
def newBoundedBlockingQueue(maxCapacity: Int): TestContext = { def newBoundedBlockingQueue(maxCapacity: Int): TestContext = {
val events: mutable.MutableList[QueueEvent] = new mutable.MutableList() val events: mutable.Buffer[QueueEvent] = new mutable.ArrayBuffer()
val realLock = new ReentrantLock(false) val realLock = new ReentrantLock(false)
val wrappedNotEmpty = new TestCondition(events, realLock.newCondition(), signalNotEmpty, awaitNotEmpty) val wrappedNotEmpty = new TestCondition(events, realLock.newCondition(), signalNotEmpty, awaitNotEmpty)

View file

@ -762,11 +762,16 @@ class ByteStringSpec extends WordSpec with Matchers with Checkers {
"serialize correctly" when { "serialize correctly" when {
"parsing regular ByteString1C as compat" in { "parsing regular ByteString1C as compat" in {
val oldSerd = "aced000573720021616b6b612e7574696c2e42797465537472696e672442797465537472696e67314336e9eed0afcfe4a40200015b000562797465737400025b427872001b616b6b612e7574696c2e436f6d7061637442797465537472696e67fa2925150f93468f0200007870757200025b42acf317f8060854e002000078700000000a74657374737472696e67" val oldSerd =
if (util.Properties.versionNumberString.startsWith("2.11") || util.Properties.versionNumberString.startsWith("2.12"))
"aced000573720021616b6b612e7574696c2e42797465537472696e672442797465537472696e67314336e9eed0afcfe4a40200015b000562797465737400025b427872001b616b6b612e7574696c2e436f6d7061637442797465537472696e67fa2925150f93468f0200007870757200025b42acf317f8060854e002000078700000000a74657374737472696e67"
else
// The data is the same, but the class hierarchy changed in 2.13:
"aced000573720021616b6b612e7574696c2e42797465537472696e672442797465537472696e67314336e9eed0afcfe4a40200015b000562797465737400025b427872001b616b6b612e7574696c2e436f6d7061637442797465537472696e676c083a30328adea002000078720014616b6b612e7574696c2e42797465537472696e678efa6cf8286d3c930200007870757200025b42acf317f8060854e002000078700000000a74657374737472696e67"
val bs = ByteString("teststring", "UTF8") val bs = ByteString("teststring", "UTF8")
val str = hexFromSer(bs) val str = hexFromSer(bs)
require(oldSerd == str) str should be(oldSerd)
} }
"given all types of ByteString" in { "given all types of ByteString" in {

View file

@ -11,7 +11,7 @@ import scala.util.Properties
class LineNumberSpec extends AkkaSpec { class LineNumberSpec extends AkkaSpec {
private val isScala212 = Properties.versionNumberString.startsWith("2.12") private val isScala211 = Properties.versionNumberString.startsWith("2.11")
"LineNumbers" when { "LineNumbers" when {
@ -24,11 +24,11 @@ class LineNumberSpec extends AkkaSpec {
"work for larger functions" in { "work for larger functions" in {
val result = LineNumbers(twoline) val result = LineNumbers(twoline)
if (isScala212) if (isScala211)
// because how scala 2.12 does the same as Java Lambdas
result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 15, 15))
else
result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 15, 17)) result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 15, 17))
else
// because how scala 2.12+ does the same as Java Lambdas
result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 15, 15))
} }
"work for partial functions" in { "work for partial functions" in {
@ -37,11 +37,11 @@ class LineNumberSpec extends AkkaSpec {
"work for `def`" in { "work for `def`" in {
val result = LineNumbers(method("foo")) val result = LineNumbers(method("foo"))
if (isScala212) if (isScala211)
result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 25, 27))
else
// because how scala 2.12 does the same as Java Lambdas // because how scala 2.12 does the same as Java Lambdas
result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 26, 27)) result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 26, 27))
else
result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 25, 27))
} }
} }

View file

@ -250,7 +250,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit(
private def nextName(prefix: String = "a"): String = s"$prefix-${nameCounter.next()}" private def nextName(prefix: String = "a"): String = s"$prefix-${nameCounter.next()}"
// FIXME eventfilter support in typed testkit // FIXME eventfilter support in typed testkit
import scaladsl.adapter._ import akka.actor.typed.scaladsl.adapter._
implicit val untypedSystem = system.toUntyped implicit val untypedSystem = system.toUntyped
@ -748,7 +748,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit(
val beh = supervise[String](setup(context val beh = supervise[String](setup(context
supervise[String](setup { context supervise[String](setup { context
probe.ref ! Started probe.ref ! Started
scaladsl.Behaviors.empty[String] Behaviors.empty[String]
}).onFailure[RuntimeException](strategy) }).onFailure[RuntimeException](strategy)
)).onFailure[Exception](strategy) )).onFailure[Exception](strategy)

View file

@ -32,18 +32,20 @@ object ReceptionistApiSpec {
// to work // to work
val registered: Future[Receptionist.Registered] = val registered: Future[Receptionist.Registered] =
system.receptionist ? (Receptionist.Register(key, service, _)) system.receptionist ? (Receptionist.Register(key, service, _))
registered.onSuccess { registered.foreach {
case key.Registered(ref) case key.Registered(ref)
// ref is the right type here // ref is the right type here
ref ! "woho" ref ! "woho"
case _ ()
} }
// one-off ask outside of actor, should be uncommon but not rare // one-off ask outside of actor, should be uncommon but not rare
val found: Future[Receptionist.Listing] = val found: Future[Receptionist.Listing] =
system.receptionist ? (Receptionist.Find(key, _)) system.receptionist ? (Receptionist.Find(key, _))
found.onSuccess { found.foreach {
case key.Listing(instances) case key.Listing(instances)
instances.foreach(_ ! "woho") instances.foreach(_ ! "woho")
case _ ()
} }
Behaviors.setup[Any] { context Behaviors.setup[Any] { context

View file

@ -6,7 +6,7 @@ package akka.actor.typed
package scaladsl package scaladsl
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
import akka.actor.typed.internal.adapter.{ PropsAdapter => _, _ } import akka.actor.typed.internal.adapter.{ PropsAdapter _, _ }
import akka.annotation.InternalApi import akka.annotation.InternalApi
/** /**

View file

@ -72,3 +72,7 @@ ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.pattern.BackoffOption
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.pattern.HandleBackoff.finalStopMessageReceived") ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.pattern.HandleBackoff.finalStopMessageReceived")
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.pattern.HandleBackoff.finalStopMessageReceived_=") ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.pattern.HandleBackoff.finalStopMessageReceived_=")
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.pattern.HandleBackoff.finalStopMessage") ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.pattern.HandleBackoff.finalStopMessage")
# Scala 2.13.0-M5
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.actor.dungeon.ChildrenContainer#ChildrenIterable.this")
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.actor.dungeon.ChildrenContainer#ChildRestartsIterable.this")

View file

@ -5,3 +5,7 @@ ProblemFilters.exclude[MissingClassProblem]("akka.io.dns.internal.AsyncDnsResolv
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.io.dns.internal.AsyncDnsCache.put") ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.io.dns.internal.AsyncDnsCache.put")
ProblemFilters.exclude[MissingClassProblem]("akka.io.dns.internal.AsyncDnsResolver$Ipv6Type$") ProblemFilters.exclude[MissingClassProblem]("akka.io.dns.internal.AsyncDnsResolver$Ipv6Type$")
ProblemFilters.exclude[MissingClassProblem]("akka.io.dns.internal.AsyncDnsResolver$Ipv4Type$") ProblemFilters.exclude[MissingClassProblem]("akka.io.dns.internal.AsyncDnsResolver$Ipv4Type$")
# Changes related to adding Scala 2.13.0-M5 support
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.actor.dungeon.ChildrenContainer#ChildrenIterable.this")
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.actor.dungeon.ChildrenContainer#ChildRestartsIterable.this")

View file

@ -1,5 +1,5 @@
/* /*
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.util package akka.util

View file

@ -7,7 +7,7 @@ package akka.compat
import akka.annotation.InternalApi import akka.annotation.InternalApi
import scala.concurrent.{ ExecutionContext, Future SFuture } import scala.concurrent.{ ExecutionContext, Future SFuture }
import scala.collection.immutable import scala.collection.immutable
import scala.collection.compat._ import akka.util.ccompat._
/** /**
* INTERNAL API * INTERNAL API

View file

@ -1,5 +1,5 @@
/* /*
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.util package akka.util

View file

@ -12,7 +12,7 @@ import scala.annotation.tailrec
import scala.collection.LinearSeq import scala.collection.LinearSeq
import scala.collection.mutable.ListBuffer import scala.collection.mutable.ListBuffer
import scala.reflect.ClassTag import scala.reflect.ClassTag
import scala.collection.compat._ import akka.util.ccompat._
object ByteIterator { object ByteIterator {
object ByteArrayIterator { object ByteArrayIterator {
@ -234,7 +234,7 @@ object ByteIterator {
} }
final override def clone: MultiByteArrayIterator = { final override def clone: MultiByteArrayIterator = {
val clonedIterators: List[ByteArrayIterator] = iterators.iterator.map(_.clone).to(scala.collection.immutable.List) val clonedIterators: List[ByteArrayIterator] = iterators.iterator.map(_.clone).to(List)
new MultiByteArrayIterator(clonedIterators) new MultiByteArrayIterator(clonedIterators)
} }
@ -391,7 +391,6 @@ object ByteIterator {
/** /**
* An iterator over a ByteString. * An iterator over a ByteString.
*/ */
abstract class ByteIterator extends BufferedIterator[Byte] { abstract class ByteIterator extends BufferedIterator[Byte] {
def len: Int def len: Int

View file

@ -1,5 +1,5 @@
/* /*
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.util package akka.util
@ -11,14 +11,13 @@ import java.nio.{ ByteBuffer, ByteOrder }
import java.lang.{ Iterable JIterable } import java.lang.{ Iterable JIterable }
import scala.annotation.{ tailrec, varargs } import scala.annotation.{ tailrec, varargs }
import scala.collection.StrictOptimizedSeqOps
import scala.collection.compat._
import scala.collection.mutable.{ Builder, WrappedArray } import scala.collection.mutable.{ Builder, WrappedArray }
import scala.collection.{ mutable, immutable } import scala.collection.{ mutable, immutable }
import scala.collection.immutable.{ IndexedSeq, IndexedSeqOps, VectorBuilder } import scala.collection.immutable.{ IndexedSeq, IndexedSeqOps, StrictOptimizedSeqOps, VectorBuilder }
import scala.collection.generic.CanBuildFrom import scala.collection.generic.CanBuildFrom
import scala.reflect.ClassTag import scala.reflect.ClassTag
import java.nio.charset.{ Charset, StandardCharsets } import java.nio.charset.{ Charset, StandardCharsets }
import akka.util.ccompat._
object ByteString { object ByteString {
@ -669,6 +668,10 @@ sealed abstract class ByteString
override protected def fromSpecific(coll: IterableOnce[Byte]): ByteString = ByteString(coll) override protected def fromSpecific(coll: IterableOnce[Byte]): ByteString = ByteString(coll)
override protected def newSpecificBuilder: mutable.Builder[Byte, ByteString] = ByteString.newBuilder override protected def newSpecificBuilder: mutable.Builder[Byte, ByteString] = ByteString.newBuilder
// FIXME this is a workaround for
// https://github.com/scala/bug/issues/11192#issuecomment-436926231
protected[this] override def writeReplace(): AnyRef = this
def apply(idx: Int): Byte def apply(idx: Int): Byte
private[akka] def byteStringCompanion: ByteString.Companion private[akka] def byteStringCompanion: ByteString.Companion
// override so that toString will also be `ByteString(...)` for the concrete subclasses // override so that toString will also be `ByteString(...)` for the concrete subclasses
@ -833,9 +836,7 @@ sealed abstract class ByteString
*/ */
final def mapI(f: Byte Int): ByteString = map(f andThen (_.toByte)) final def mapI(f: Byte Int): ByteString = map(f andThen (_.toByte))
def map[A](f: Byte Byte): ByteString = { def map[A](f: Byte Byte): ByteString = fromSpecific(super.map(f))
???
}
} }
object CompactByteString { object CompactByteString {

View file

@ -1,5 +1,5 @@
/* /*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2018-2019 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.util.ccompat package akka.util.ccompat
@ -7,5 +7,4 @@ package akka.util.ccompat
import scala.collection.immutable import scala.collection.immutable
package object imm { package object imm {
type MapLike[K, +V, +This <: immutable.Map[K, V]] = immutable.Map[K, V]
} }

View file

@ -0,0 +1,17 @@
/*
* Copyright (C) 2018-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.util
/**
* INTERNAL API
*
* Based on https://github.com/scala/scala-collection-compat/blob/master/compat/src/main/scala-2.13/scala/collection/compat/package.scala
* but reproduced here so we don't need to add a dependency on this library. It contains much more than we need right now, and is
* not promising binary compatibility yet at the time of writing.
*/
package object ccompat {
private[akka] type Factory[-A, +C] = scala.collection.Factory[A, C]
private[akka] val Factory = scala.collection.Factory
}

View file

@ -0,0 +1,22 @@
/*
* Copyright (C) 2018-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.util.ccompat
import scala.collection.generic.CanBuildFrom
import scala.collection.mutable.Builder
/**
* INTERNAL API
*
* Based on https://github.com/scala/scala-collection-compat/blob/master/compat/src/main/scala-2.11_2.12/scala/collection/compat/CompatImpl.scala
* but reproduced here so we don't need to add a dependency on this library. It contains much more than we need right now, and is
* not promising binary compatibility yet at the time of writing.
*/
private[ccompat] object CompatImpl {
def simpleCBF[A, C](f: Builder[A, C]): CanBuildFrom[Any, A, C] = new CanBuildFrom[Any, A, C] {
def apply(from: Any): Builder[A, C] = apply()
def apply(): Builder[A, C] = f
}
}

View file

@ -1,11 +1,16 @@
/* /*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2018-2019 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.util.ccompat package akka.util.ccompat
import scala.collection.immutable import scala.collection.immutable
/**
* INTERNAL API
*/
package object imm { package object imm {
type MapLike[K, +V, +This <: MapLike[K, V, This] with immutable.Map[K, V]] = immutable.MapLike[K, V, This] implicit class SortedSetOps[A](val real: immutable.SortedSet[A]) extends AnyVal {
def unsorted: immutable.Set[A] = real
}
} }

View file

@ -0,0 +1,86 @@
/*
* Copyright (C) 2018-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.util
import scala.language.implicitConversions
import scala.language.higherKinds
import scala.collection.GenTraversable
import scala.{ collection c }
import scala.collection.generic.{ CanBuildFrom, GenericCompanion, Sorted, SortedSetFactory }
import scala.collection.{ immutable i }
import scala.collection.{ mutable m }
/**
* INTERNAL API
*
* Based on https://github.com/scala/scala-collection-compat/blob/master/compat/src/main/scala-2.11_2.12/scala/collection/compat/PackageShared.scala
* but reproduced here so we don't need to add a dependency on this library. It contains much more than we need right now, and is
* not promising binary compatibility yet at the time of writing.
*/
package object ccompat {
import CompatImpl._
/**
* A factory that builds a collection of type `C` with elements of type `A`.
*
* @tparam A Type of elements (e.g. `Int`, `Boolean`, etc.)
* @tparam C Type of collection (e.g. `List[Int]`, `TreeMap[Int, String]`, etc.)
*/
private[akka] type Factory[-A, +C] <: CanBuildFrom[Nothing, A, C] // Ideally, this would be an opaque type
private[akka] implicit class FactoryOps[-A, +C](private val factory: Factory[A, C]) {
/**
* @return A collection of type `C` containing the same elements
* as the source collection `it`.
* @param it Source collection
*/
def fromSpecific(it: TraversableOnce[A]): C = (factory() ++= it).result()
/**
* Get a Builder for the collection. For non-strict collection types this will use an intermediate buffer.
* Building collections with `fromSpecific` is preferred because it can be lazy for lazy collections.
*/
def newBuilder: m.Builder[A, C] = factory()
}
private[akka] implicit def fromCanBuildFrom[A, C](implicit cbf: CanBuildFrom[Nothing, A, C]): Factory[A, C] =
cbf.asInstanceOf[Factory[A, C]]
private[akka] implicit def genericCompanionToCBF[A, CC[X] <: GenTraversable[X]](
fact: GenericCompanion[CC]): CanBuildFrom[Any, A, CC[A]] =
simpleCBF(fact.newBuilder[A])
private[akka] implicit def sortedSetCompanionToCBF[A: Ordering, CC[X] <: c.SortedSet[X] with c.SortedSetLike[X, CC[X]]](
fact: SortedSetFactory[CC]): CanBuildFrom[Any, A, CC[A]] =
simpleCBF(fact.newBuilder[A])
private[ccompat] def build[T, CC](builder: m.Builder[T, CC], source: TraversableOnce[T]): CC = {
builder ++= source
builder.result()
}
private[akka] implicit class ImmutableSortedMapExtensions(private val fact: i.SortedMap.type) extends AnyVal {
def from[K: Ordering, V](source: TraversableOnce[(K, V)]): i.SortedMap[K, V] =
build(i.SortedMap.newBuilder[K, V], source)
}
private[akka] implicit class ImmutableTreeMapExtensions(private val fact: i.TreeMap.type) extends AnyVal {
def from[K: Ordering, V](source: TraversableOnce[(K, V)]): i.TreeMap[K, V] =
build(i.TreeMap.newBuilder[K, V], source)
}
private[akka] implicit class SortedExtensionMethods[K, T <: Sorted[K, T]](private val fact: Sorted[K, T]) {
def rangeFrom(from: K): T = fact.from(from)
def rangeTo(to: K): T = fact.to(to)
def rangeUntil(until: K): T = fact.until(until)
}
// This really belongs into scala.collection but there's already a package object
// in scala-library so we can't add to it
type IterableOnce[+X] = c.TraversableOnce[X]
val IterableOnce = c.TraversableOnce
}

View file

@ -1,5 +1,5 @@
/* /*
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com> * Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
*/ */
package akka.compat package akka.compat

View file

@ -22,7 +22,7 @@ import akka.util.{ Helpers, JavaDurationConverters, Timeout }
import akka.dispatch.ExecutionContexts import akka.dispatch.ExecutionContexts
import scala.compat.java8.FutureConverters import scala.compat.java8.FutureConverters
import scala.collection.compat._ import akka.util.ccompat._
/** /**
* An ActorSelection is a logical view of a section of an ActorSystem's tree of Actors, * An ActorSelection is a logical view of a section of an ActorSystem's tree of Actors,
@ -209,7 +209,7 @@ object ActorSelection {
if ((x.indexOf('?') != -1) || (x.indexOf('*') != -1)) SelectChildPattern(x) if ((x.indexOf('?') != -1) || (x.indexOf('*') != -1)) SelectChildPattern(x)
else if (x == "..") SelectParent else if (x == "..") SelectParent
else SelectChildName(x) else SelectChildName(x)
}).to(scala.collection.immutable.IndexedSeq) }).to(immutable.IndexedSeq)
new ActorSelection with ScalaActorSelection { new ActorSelection with ScalaActorSelection {
override val anchor = anchorRef override val anchor = anchorRef
override val path = compiled override val path = compiled

View file

@ -17,7 +17,7 @@ import akka.event.Logging.LogEvent
import akka.event.Logging.Error import akka.event.Logging.Error
import akka.event.Logging.Warning import akka.event.Logging.Warning
import scala.util.control.NonFatal import scala.util.control.NonFatal
import scala.collection.compat._ import akka.util.ccompat._
/** /**
* INTERNAL API * INTERNAL API

View file

@ -8,7 +8,6 @@ import scala.collection.immutable
import akka.actor.{ InvalidActorNameException, ChildStats, ChildRestartStats, ChildNameReserved, ActorRef } import akka.actor.{ InvalidActorNameException, ChildStats, ChildRestartStats, ChildNameReserved, ActorRef }
import akka.util.Collections.{ EmptyImmutableSeq, PartialImmutableValuesIterable } import akka.util.Collections.{ EmptyImmutableSeq, PartialImmutableValuesIterable }
import akka.util.ccompat
/** /**
* INTERNAL API * INTERNAL API
@ -50,13 +49,13 @@ private[akka] object ChildrenContainer {
final case class Creation() extends SuspendReason with WaitingForChildren final case class Creation() extends SuspendReason with WaitingForChildren
case object Termination extends SuspendReason case object Termination extends SuspendReason
class ChildRestartsIterable(stats: ccompat.imm.MapLike[_, ChildStats, _]) extends PartialImmutableValuesIterable[ChildStats, ChildRestartStats] { class ChildRestartsIterable(stats: immutable.Map[_, ChildStats]) extends PartialImmutableValuesIterable[ChildStats, ChildRestartStats] {
override final def apply(c: ChildStats) = c.asInstanceOf[ChildRestartStats] override final def apply(c: ChildStats) = c.asInstanceOf[ChildRestartStats]
override final def isDefinedAt(c: ChildStats) = c.isInstanceOf[ChildRestartStats] override final def isDefinedAt(c: ChildStats) = c.isInstanceOf[ChildRestartStats]
override final def valuesIterator = stats.valuesIterator override final def valuesIterator = stats.valuesIterator
} }
class ChildrenIterable(stats: ccompat.imm.MapLike[_, ChildStats, _]) extends PartialImmutableValuesIterable[ChildStats, ActorRef] { class ChildrenIterable(stats: immutable.Map[_, ChildStats]) extends PartialImmutableValuesIterable[ChildStats, ActorRef] {
override final def apply(c: ChildStats) = c.asInstanceOf[ChildRestartStats].child override final def apply(c: ChildStats) = c.asInstanceOf[ChildRestartStats].child
override final def isDefinedAt(c: ChildStats) = c.isInstanceOf[ChildRestartStats] override final def isDefinedAt(c: ChildStats) = c.isInstanceOf[ChildRestartStats]
override final def valuesIterator = stats.valuesIterator override final def valuesIterator = stats.valuesIterator

View file

@ -16,7 +16,7 @@ import java.util.function.{ Function ⇒ JFunction }
import akka.util.unused import akka.util.unused
import scala.collection.immutable import scala.collection.immutable
import scala.collection.compat._ import akka.util.ccompat._
abstract class Dns { abstract class Dns {
@ -59,10 +59,10 @@ object Dns extends ExtensionId[DnsExt] with ExtensionIdProvider {
def apply(name: String, addresses: Iterable[InetAddress]): Resolved = { def apply(name: String, addresses: Iterable[InetAddress]): Resolved = {
val ipv4: immutable.Seq[Inet4Address] = addresses.iterator.collect({ val ipv4: immutable.Seq[Inet4Address] = addresses.iterator.collect({
case a: Inet4Address a case a: Inet4Address a
}).to(scala.collection.immutable.IndexedSeq) }).to(immutable.IndexedSeq)
val ipv6: immutable.Seq[Inet6Address] = addresses.iterator.collect({ val ipv6: immutable.Seq[Inet6Address] = addresses.iterator.collect({
case a: Inet6Address a case a: Inet6Address a
}).to(scala.collection.immutable.IndexedSeq) }).to(immutable.IndexedSeq)
Resolved(name, ipv4, ipv6) Resolved(name, ipv4, ipv6)
} }
} }

View file

@ -21,7 +21,6 @@ import java.lang.{ Iterable ⇒ JIterable }
import java.nio.file.Path import java.nio.file.Path
import akka.annotation.InternalApi import akka.annotation.InternalApi
import scala.collection.immutable
/** /**
* TCP Extension for Akkas IO layer. * TCP Extension for Akkas IO layer.

View file

@ -22,7 +22,6 @@ import scala.annotation.tailrec
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.util.control.{ NoStackTrace, NonFatal } import scala.util.control.{ NoStackTrace, NonFatal }
import scala.collection.immutable
/** /**
* Base class for TcpIncomingConnection and TcpOutgoingConnection. * Base class for TcpIncomingConnection and TcpOutgoingConnection.

View file

@ -8,7 +8,6 @@ import java.nio.channels.SocketChannel
import scala.collection.immutable import scala.collection.immutable
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.io.Inet.SocketOption import akka.io.Inet.SocketOption
import scala.collection.immutable
/** /**
* An actor handling the connection state machine for an incoming, already connected * An actor handling the connection state machine for an incoming, already connected

View file

@ -12,8 +12,7 @@ import akka.io.Inet.{ SoJavaFactories, SocketOption }
import akka.util.Helpers.Requiring import akka.util.Helpers.Requiring
import akka.util.ByteString import akka.util.ByteString
import akka.actor._ import akka.actor._
import scala.collection.immutable import akka.util.ccompat._
import scala.collection.compat._
/** /**
* UDP Extension for Akkas IO layer. * UDP Extension for Akkas IO layer.
@ -282,7 +281,7 @@ object UdpMessage {
* message, or the manager will reply with a [[Udp.CommandFailed]] message. * message, or the manager will reply with a [[Udp.CommandFailed]] message.
*/ */
def bind(handler: ActorRef, endpoint: InetSocketAddress, options: JIterable[SocketOption]): Command = def bind(handler: ActorRef, endpoint: InetSocketAddress, options: JIterable[SocketOption]): Command =
Bind(handler, endpoint, options.asScala.to(scala.collection.immutable.IndexedSeq)) Bind(handler, endpoint, options.asScala.to(immutable.IndexedSeq))
/** /**
* Bind without specifying options. * Bind without specifying options.
*/ */
@ -305,7 +304,7 @@ object UdpMessage {
* The simple sender will not stop itself, you will have to send it a [[akka.actor.PoisonPill]] * The simple sender will not stop itself, you will have to send it a [[akka.actor.PoisonPill]]
* when you want to close the socket. * when you want to close the socket.
*/ */
def simpleSender(options: JIterable[SocketOption]): Command = SimpleSender(options.asScala.to(scala.collection.immutable.IndexedSeq)) def simpleSender(options: JIterable[SocketOption]): Command = SimpleSender(options.asScala.to(immutable.IndexedSeq))
/** /**
* Retrieve a simple sender without specifying options. * Retrieve a simple sender without specifying options.
*/ */

View file

@ -11,8 +11,7 @@ import akka.io.Inet.SocketOption
import akka.io.Udp.UdpSettings import akka.io.Udp.UdpSettings
import akka.util.ByteString import akka.util.ByteString
import akka.actor._ import akka.actor._
import scala.collection.immutable import akka.util.ccompat._
import scala.collection.compat._
/** /**
* UDP Extension for Akkas IO layer. * UDP Extension for Akkas IO layer.

View file

@ -11,7 +11,6 @@ import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
import akka.io.Inet.SocketOption import akka.io.Inet.SocketOption
import akka.io.Udp._ import akka.io.Udp._
import akka.actor._ import akka.actor._
import scala.collection.immutable
/** /**
* INTERNAL API * INTERNAL API

View file

@ -19,7 +19,7 @@ import scala.collection.JavaConverters._
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.duration.FiniteDuration import scala.concurrent.duration.FiniteDuration
import scala.util.{ Failure, Success, Try } import scala.util.{ Failure, Success, Try }
import scala.collection.compat._ import akka.util.ccompat._
/** INTERNAL API */ /** INTERNAL API */
@InternalApi @InternalApi
@ -39,7 +39,7 @@ private[dns] final class DnsSettings(system: ExtendedActorSystem, c: Config) {
parseNameserverAddress(other) :: Nil parseNameserverAddress(other) :: Nil
} }
case ConfigValueType.LIST case ConfigValueType.LIST
val userAddresses = c.getStringList("nameservers").asScala.iterator.map(parseNameserverAddress).to(scala.collection.immutable.IndexedSeq) val userAddresses = c.getStringList("nameservers").asScala.iterator.map(parseNameserverAddress).to(immutable.IndexedSeq)
require(userAddresses.nonEmpty, "nameservers can not be empty") require(userAddresses.nonEmpty, "nameservers can not be empty")
userAddresses.toList userAddresses.toList
case _ throw new IllegalArgumentException("Invalid type for nameservers. Must be a string or string list") case _ throw new IllegalArgumentException("Invalid type for nameservers. Must be a string or string list")

View file

@ -92,10 +92,7 @@ private[io] final class AsyncDnsResolver(
private def sendQuestion(resolver: ActorRef, message: DnsQuestion): Future[Answer] = { private def sendQuestion(resolver: ActorRef, message: DnsQuestion): Future[Answer] = {
val result = (resolver ? message).mapTo[Answer] val result = (resolver ? message).mapTo[Answer]
result.onComplete { result.failed.foreach { _ resolver ! DropRequest(message.id) }
case scala.util.Failure(NonFatal(_)) resolver ! DropRequest(message.id)
case _ ()
}
result result
} }

View file

@ -21,7 +21,7 @@
package akka.routing package akka.routing
import java.lang.Integer.{ rotateLeft rotl } import java.lang.Integer.{ rotateLeft rotl }
import scala.collection.compat._ import akka.util.ccompat._
/** /**
* An object designed to generate well-distributed non-cryptographic * An object designed to generate well-distributed non-cryptographic

View file

@ -20,7 +20,7 @@ import akka.dispatch.MessageDispatcher
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.collection.compat._ import akka.util.ccompat._
/** /**
* INTERNAL API * INTERNAL API
@ -110,7 +110,7 @@ private[akka] class RoutedActorCell(
case group: Group case group: Group
val paths = group.paths(system) val paths = group.paths(system)
if (paths.nonEmpty) if (paths.nonEmpty)
addRoutees(paths.iterator.map(p group.routeeFor(p, this)).to(scala.collection.immutable.IndexedSeq)) addRoutees(paths.iterator.map(p group.routeeFor(p, this)).to(immutable.IndexedSeq))
case _ case _
} }
preSuperStart() preSuperStart()

View file

@ -22,7 +22,7 @@ import java.util.concurrent.atomic.AtomicReference
import scala.annotation.tailrec import scala.annotation.tailrec
import java.util.NoSuchElementException import java.util.NoSuchElementException
import akka.annotation.InternalApi import akka.annotation.InternalApi
import scala.collection.compat._ import akka.util.ccompat._
object Serialization { object Serialization {
@ -376,6 +376,8 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
system.dynamicAccess.createInstanceFor[Serializer](fqn, List(classOf[ExtendedActorSystem] system)) recoverWith { system.dynamicAccess.createInstanceFor[Serializer](fqn, List(classOf[ExtendedActorSystem] system)) recoverWith {
case _: NoSuchMethodException case _: NoSuchMethodException
system.dynamicAccess.createInstanceFor[Serializer](fqn, Nil) system.dynamicAccess.createInstanceFor[Serializer](fqn, Nil)
// FIXME only needed on 2.13.0-M5 due to https://github.com/scala/bug/issues/11242
case t Failure(t)
} }
} }

View file

@ -5,7 +5,7 @@
package akka.util package akka.util
import scala.collection.immutable import scala.collection.immutable
import scala.collection.compat._ import akka.util.ccompat._
/** /**
* Typeclass which describes a classification hierarchy. Observe the contract between `isEqual` and `isSubclass`! * Typeclass which describes a classification hierarchy. Observe the contract between `isEqual` and `isSubclass`!

View file

@ -95,7 +95,7 @@ class CamelSettings private[camel] (config: Config, dynamicAccess: DynamicAccess
val section = config.getConfig("akka.camel.conversions") val section = config.getConfig("akka.camel.conversions")
section.entrySet.asScala.map(e (e.getKey, section.getString(e.getKey))) section.entrySet.asScala.map(e (e.getKey, section.getString(e.getKey)))
} }
val conversions = (Map[String, Class[_ <: AnyRef]]() /: specifiedConversions) { val conversions = specifiedConversions.foldLeft(Map[String, Class[_ <: AnyRef]]()) {
case (m, (key, fqcn)) case (m, (key, fqcn))
m.updated(key, dynamicAccess.getClassFor[AnyRef](fqcn).recover { m.updated(key, dynamicAccess.getClassFor[AnyRef](fqcn).recover {
case e throw new ConfigurationException("Could not find/load Camel Converter class [" + fqcn + "]", e) case e throw new ConfigurationException("Could not find/load Camel Converter class [" + fqcn + "]", e)

View file

@ -11,16 +11,16 @@ import akka.AkkaException
import scala.reflect.ClassTag import scala.reflect.ClassTag
import scala.runtime.ScalaRunTime import scala.runtime.ScalaRunTime
import scala.util.Try import scala.util.Try
import scala.collection.JavaConversions._
import akka.dispatch.Mapper import akka.dispatch.Mapper
import scala.collection.JavaConverters._
/** /**
* An immutable representation of a Camel message. * An immutable representation of a Camel message.
*/ */
@deprecated("Akka Camel is deprecated in favour of 'Alpakka', the Akka Streams based collection of integrations to various endpoints (including Camel).", since = "2.5.0") @deprecated("Akka Camel is deprecated in favour of 'Alpakka', the Akka Streams based collection of integrations to various endpoints (including Camel).", since = "2.5.0")
class CamelMessage(val body: Any, val headers: Map[String, Any], val attachments: Map[String, DataHandler]) extends Serializable with Product { class CamelMessage(val body: Any, val headers: Map[String, Any], val attachments: Map[String, DataHandler]) extends Serializable with Product {
def this(body: Any, headers: JMap[String, Any]) = this(body, headers.toMap, Map.empty[String, DataHandler]) //Java def this(body: Any, headers: JMap[String, Any]) = this(body, headers.asScala.toMap, Map.empty[String, DataHandler]) //Java
def this(body: Any, headers: JMap[String, Any], attachments: JMap[String, DataHandler]) = this(body, headers.toMap, attachments.toMap) //Java def this(body: Any, headers: JMap[String, Any], attachments: JMap[String, DataHandler]) = this(body, headers.asScala.toMap, attachments.asScala.toMap) //Java
def this(body: Any, headers: Map[String, Any]) = this(body, headers.toMap, Map.empty[String, DataHandler]) def this(body: Any, headers: Map[String, Any]) = this(body, headers.toMap, Map.empty[String, DataHandler])
def copy(body: Any = this.body, headers: Map[String, Any] = this.headers): CamelMessage = CamelMessage(body, headers, this.attachments) def copy(body: Any = this.body, headers: Map[String, Any] = this.headers): CamelMessage = CamelMessage(body, headers, this.attachments)
@ -37,19 +37,19 @@ class CamelMessage(val body: Any, val headers: Map[String, Any], val attachments
* The returned headers map is backed up by an immutable headers map. Any attempt to modify * The returned headers map is backed up by an immutable headers map. Any attempt to modify
* the returned map will throw an exception. * the returned map will throw an exception.
*/ */
def getHeaders(names: JSet[String]): JMap[String, Any] = headers(names.toSet) def getHeaders(names: JSet[String]): JMap[String, Any] = headers(names.asScala.toSet).asJava
/** /**
* Java API: Returns all headers from this message. The returned headers map is backed up by this * Java API: Returns all headers from this message. The returned headers map is backed up by this
* message's immutable headers map. Any attempt to modify the returned map will throw an * message's immutable headers map. Any attempt to modify the returned map will throw an
* exception. * exception.
*/ */
def getHeaders: JMap[String, Any] = headers def getHeaders: JMap[String, Any] = headers.asJava
/** /**
* Java API: Creates a new CamelMessage with given <code>headers</code>. A copy of the headers map is made. * Java API: Creates a new CamelMessage with given <code>headers</code>. A copy of the headers map is made.
*/ */
def withHeaders[A](headers: JMap[String, A]): CamelMessage = copy(this.body, headers.toMap) def withHeaders[A](headers: JMap[String, A]): CamelMessage = copy(this.body, headers.asScala.toMap)
/** /**
* Returns the header by given <code>name</code> parameter in a [[scala.util.Try]]. The header is converted to type <code>T</code>, which is returned * Returns the header by given <code>name</code> parameter in a [[scala.util.Try]]. The header is converted to type <code>T</code>, which is returned
@ -145,19 +145,19 @@ class CamelMessage(val body: Any, val headers: Map[String, Any], val attachments
* The returned headers map is backed up by an immutable headers map. Any attempt to modify * The returned headers map is backed up by an immutable headers map. Any attempt to modify
* the returned map will throw an exception. * the returned map will throw an exception.
*/ */
def getAttachments(names: JSet[String]): JMap[String, DataHandler] = attachments(names.toSet) def getAttachments(names: JSet[String]): JMap[String, DataHandler] = attachments(names.asScala.toSet).asJava
/** /**
* Java API: Returns all attachments from this message. The returned attachments map is backed up by this * Java API: Returns all attachments from this message. The returned attachments map is backed up by this
* message's immutable headers map. Any attempt to modify the returned map will throw an * message's immutable headers map. Any attempt to modify the returned map will throw an
* exception. * exception.
*/ */
def getAttachments: JMap[String, DataHandler] = attachments def getAttachments: JMap[String, DataHandler] = attachments.asJava
/** /**
* Java API: Creates a new CamelMessage with given <code>attachments</code>. A copy of the attachments map is made. * Java API: Creates a new CamelMessage with given <code>attachments</code>. A copy of the attachments map is made.
*/ */
def withAttachments(attachments: JMap[String, DataHandler]): CamelMessage = CamelMessage(this.body, this.headers, attachments.toMap) def withAttachments(attachments: JMap[String, DataHandler]): CamelMessage = CamelMessage(this.body, this.headers, attachments.asScala.toMap)
/** /**
* SCALA API: Creates a new CamelMessage with given <code>attachments</code>. * SCALA API: Creates a new CamelMessage with given <code>attachments</code>.
@ -250,7 +250,7 @@ object CamelMessage extends ((Any, Map[String, Any]) ⇒ CamelMessage) {
* in the Camel message. * in the Camel message.
*/ */
private[camel] def from(camelMessage: JCamelMessage, headers: Map[String, Any]): CamelMessage = private[camel] def from(camelMessage: JCamelMessage, headers: Map[String, Any]): CamelMessage =
CamelMessage(camelMessage.getBody, headers ++ camelMessage.getHeaders, camelMessage.getAttachments.toMap) CamelMessage(camelMessage.getBody, headers ++ camelMessage.getHeaders.asScala, camelMessage.getAttachments.asScala.toMap)
/** /**
* Creates a new CamelMessageWithAttachments object from the Camel message. * Creates a new CamelMessageWithAttachments object from the Camel message.
@ -261,7 +261,7 @@ object CamelMessage extends ((Any, Map[String, Any]) ⇒ CamelMessage) {
* in the Camel message. * in the Camel message.
*/ */
private[camel] def from(camelMessage: JCamelMessage, headers: Map[String, Any], attachments: Map[String, DataHandler]): CamelMessage = private[camel] def from(camelMessage: JCamelMessage, headers: Map[String, Any], attachments: Map[String, DataHandler]): CamelMessage =
CamelMessage(camelMessage.getBody, headers ++ camelMessage.getHeaders, attachments ++ camelMessage.getAttachments) CamelMessage(camelMessage.getBody, headers ++ camelMessage.getHeaders.asScala, attachments ++ camelMessage.getAttachments.asScala)
/** /**
* INTERNAL API * INTERNAL API

View file

@ -368,7 +368,8 @@ abstract class MixMetricsSelectorBase(selectors: immutable.IndexedSeq[CapacityMe
override def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] = { override def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] = {
val combined: immutable.IndexedSeq[(Address, Double)] = selectors.flatMap(_.capacity(nodeMetrics).toSeq) val combined: immutable.IndexedSeq[(Address, Double)] = selectors.flatMap(_.capacity(nodeMetrics).toSeq)
// aggregated average of the capacities by address // aggregated average of the capacities by address
combined.foldLeft(Map.empty[Address, (Double, Int)].withDefaultValue((0.0, 0))) { val init: Map[Address, (Double, Int)] = Map.empty.withDefaultValue((0.0, 0))
combined.foldLeft(init) {
case (acc, (address, capacity)) case (acc, (address, capacity))
val (sum, count) = acc(address) val (sum, count) = acc(address)
acc + (address ((sum + capacity, count + 1))) acc + (address ((sum + capacity, count + 1)))

View file

@ -282,8 +282,8 @@ final case class NodeMetrics(address: Address, timestamp: Long, metrics: Set[Met
require(address == that.address, s"merge only allowed for same address, [$address] != [$that.address]") require(address == that.address, s"merge only allowed for same address, [$address] != [$that.address]")
if (timestamp >= that.timestamp) this // that is older if (timestamp >= that.timestamp) this // that is older
else { else {
// equality is based on the name of the Metric and Set doesn't replace existing element // equality is based on the name of the Metric
copy(metrics = that.metrics union metrics, timestamp = that.timestamp) copy(metrics = that.metrics union (metrics diff that.metrics), timestamp = that.timestamp)
} }
} }
@ -303,8 +303,8 @@ final case class NodeMetrics(address: Address, timestamp: Long, metrics: Set[Met
current :+ latest current :+ latest
} }
// Append metrics missing from either latest or current. // Append metrics missing from either latest or current.
// Equality is based on the [[Metric.name]] and [[Set]] doesn't replace existing elements. // Equality is based on the [[Metric.name]]
val merged = updated union latestNode.metrics union currentNode.metrics val merged = updated union (latestNode.metrics diff updated) union (currentNode.metrics diff updated diff latestNode.metrics)
copy(metrics = merged, timestamp = latestNode.timestamp) copy(metrics = merged, timestamp = latestNode.timestamp)
} }

View file

@ -14,8 +14,10 @@ import akka.cluster.metrics._
import akka.serialization.{ BaseSerializer, SerializationExtension, Serializers, SerializerWithStringManifest } import akka.serialization.{ BaseSerializer, SerializationExtension, Serializers, SerializerWithStringManifest }
import akka.util.ClassLoaderObjectInputStream import akka.util.ClassLoaderObjectInputStream
import akka.protobuf.{ ByteString, MessageLite } import akka.protobuf.{ ByteString, MessageLite }
import akka.util.ccompat._
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.collection.immutable
import scala.collection.JavaConverters.{ asJavaIterableConverter, asScalaBufferConverter, setAsJavaSetConverter } import scala.collection.JavaConverters.{ asJavaIterableConverter, asScalaBufferConverter, setAsJavaSetConverter }
import java.io.NotSerializableException import java.io.NotSerializableException
@ -170,9 +172,8 @@ class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithS
} }
private def metricsGossipEnvelopeToProto(envelope: MetricsGossipEnvelope): cm.MetricsGossipEnvelope = { private def metricsGossipEnvelopeToProto(envelope: MetricsGossipEnvelope): cm.MetricsGossipEnvelope = {
import scala.collection.breakOut
val allNodeMetrics = envelope.gossip.nodes val allNodeMetrics = envelope.gossip.nodes
val allAddresses: Vector[Address] = allNodeMetrics.map(_.address)(breakOut) val allAddresses: Vector[Address] = allNodeMetrics.iterator.map(_.address).to(immutable.Vector)
val addressMapping = allAddresses.zipWithIndex.toMap val addressMapping = allAddresses.zipWithIndex.toMap
val allMetricNames: Vector[String] = allNodeMetrics.foldLeft(Set.empty[String])((s, n) s ++ n.metrics.iterator.map(_.name)).toVector val allMetricNames: Vector[String] = allNodeMetrics.foldLeft(Set.empty[String])((s, n) s ++ n.metrics.iterator.map(_.name)).toVector
val metricNamesMapping = allMetricNames.zipWithIndex.toMap val metricNamesMapping = allMetricNames.zipWithIndex.toMap
@ -221,9 +222,8 @@ class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithS
metricsGossipEnvelopeFromProto(cm.MetricsGossipEnvelope.parseFrom(decompress(bytes))) metricsGossipEnvelopeFromProto(cm.MetricsGossipEnvelope.parseFrom(decompress(bytes)))
private def metricsGossipEnvelopeFromProto(envelope: cm.MetricsGossipEnvelope): MetricsGossipEnvelope = { private def metricsGossipEnvelopeFromProto(envelope: cm.MetricsGossipEnvelope): MetricsGossipEnvelope = {
import scala.collection.breakOut
val mgossip = envelope.getGossip val mgossip = envelope.getGossip
val addressMapping: Vector[Address] = mgossip.getAllAddressesList.asScala.map(addressFromProto)(breakOut) val addressMapping: Vector[Address] = mgossip.getAllAddressesList.asScala.iterator.map(addressFromProto).to(Vector)
val metricNameMapping: Vector[String] = mgossip.getAllMetricNamesList.asScala.toVector val metricNameMapping: Vector[String] = mgossip.getAllMetricNamesList.asScala.toVector
def ewmaFromProto(ewma: cm.NodeMetrics.EWMA): Option[EWMA] = def ewmaFromProto(ewma: cm.NodeMetrics.EWMA): Option[EWMA] =
@ -252,9 +252,9 @@ class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithS
def nodeMetricsFromProto(nodeMetrics: cm.NodeMetrics): NodeMetrics = def nodeMetricsFromProto(nodeMetrics: cm.NodeMetrics): NodeMetrics =
NodeMetrics(addressMapping(nodeMetrics.getAddressIndex), nodeMetrics.getTimestamp, NodeMetrics(addressMapping(nodeMetrics.getAddressIndex), nodeMetrics.getTimestamp,
nodeMetrics.getMetricsList.asScala.map(metricFromProto)(breakOut)) nodeMetrics.getMetricsList.asScala.iterator.map(metricFromProto).to(immutable.Set))
val nodeMetrics: Set[NodeMetrics] = mgossip.getNodeMetricsList.asScala.map(nodeMetricsFromProto)(breakOut) val nodeMetrics: Set[NodeMetrics] = mgossip.getNodeMetricsList.asScala.iterator.map(nodeMetricsFromProto).to(immutable.Set)
MetricsGossipEnvelope(addressFromProto(envelope.getFrom), MetricsGossip(nodeMetrics), envelope.getReply) MetricsGossipEnvelope(addressFromProto(envelope.getFrom), MetricsGossip(nodeMetrics), envelope.getReply)
} }

View file

@ -31,6 +31,7 @@ import akka.serialization.SerializerWithStringManifest
import akka.util.Timeout import akka.util.Timeout
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
import akka.util.ccompat.imm._
object ClusterShardingSpec { object ClusterShardingSpec {
val config = ConfigFactory.parseString( val config = ConfigFactory.parseString(
@ -236,11 +237,11 @@ class ClusterShardingSpec extends ScalaTestWithActorTestKit(ClusterShardingSpec.
Cluster(system2).manager ! Join(Cluster(system).selfMember.address) Cluster(system2).manager ! Join(Cluster(system).selfMember.address)
eventually { eventually {
Cluster(system).state.members.map(_.status) should ===(Set[MemberStatus](MemberStatus.Up)) Cluster(system).state.members.unsorted.map(_.status) should ===(Set[MemberStatus](MemberStatus.Up))
Cluster(system).state.members.size should ===(2) Cluster(system).state.members.size should ===(2)
} }
eventually { eventually {
Cluster(system2).state.members.map(_.status) should ===(Set[MemberStatus](MemberStatus.Up)) Cluster(system2).state.members.unsorted.map(_.status) should ===(Set[MemberStatus](MemberStatus.Up))
Cluster(system2).state.members.size should ===(2) Cluster(system2).state.members.size should ===(2)
} }

View file

@ -10,7 +10,7 @@ import java.util.zip.GZIPOutputStream
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
import scala.collection.breakOut import scala.collection.immutable
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
import akka.cluster.sharding.Shard import akka.cluster.sharding.Shard
@ -20,6 +20,7 @@ import akka.serialization.BaseSerializer
import akka.serialization.Serialization import akka.serialization.Serialization
import akka.serialization.SerializerWithStringManifest import akka.serialization.SerializerWithStringManifest
import akka.protobuf.MessageLite import akka.protobuf.MessageLite
import akka.util.ccompat._
import java.io.NotSerializableException import java.io.NotSerializableException
import akka.cluster.sharding.ShardRegion._ import akka.cluster.sharding.ShardRegion._
@ -214,16 +215,16 @@ private[akka] class ClusterShardingMessageSerializer(val system: ExtendedActorSy
private def coordinatorStateFromProto(state: sm.CoordinatorState): State = { private def coordinatorStateFromProto(state: sm.CoordinatorState): State = {
val shards: Map[String, ActorRef] = val shards: Map[String, ActorRef] =
state.getShardsList.asScala.toVector.map { entry state.getShardsList.asScala.toVector.iterator.map { entry
entry.getShardId resolveActorRef(entry.getRegionRef) entry.getShardId resolveActorRef(entry.getRegionRef)
}(breakOut) }.toMap
val regionsZero: Map[ActorRef, Vector[String]] = val regionsZero: Map[ActorRef, Vector[String]] =
state.getRegionsList.asScala.toVector.map(resolveActorRef(_) Vector.empty[String])(breakOut) state.getRegionsList.asScala.toVector.iterator.map(resolveActorRef(_) Vector.empty[String]).toMap
val regions: Map[ActorRef, Vector[String]] = val regions: Map[ActorRef, Vector[String]] =
shards.foldLeft(regionsZero) { case (acc, (shardId, regionRef)) acc.updated(regionRef, acc(regionRef) :+ shardId) } shards.foldLeft(regionsZero) { case (acc, (shardId, regionRef)) acc.updated(regionRef, acc(regionRef) :+ shardId) }
val proxies: Set[ActorRef] = state.getRegionProxiesList.asScala.map { resolveActorRef }(breakOut) val proxies: Set[ActorRef] = state.getRegionProxiesList.asScala.iterator.map { resolveActorRef }.to(immutable.Set)
val unallocatedShards: Set[String] = state.getUnallocatedShardsList.asScala.toSet val unallocatedShards: Set[String] = state.getUnallocatedShardsList.asScala.toSet
State(shards, regions, proxies, unallocatedShards) State(shards, regions, proxies, unallocatedShards)
@ -299,7 +300,7 @@ private[akka] class ClusterShardingMessageSerializer(val system: ExtendedActorSy
private def shardRegionStatsFromBinary(bytes: Array[Byte]): ShardRegionStats = { private def shardRegionStatsFromBinary(bytes: Array[Byte]): ShardRegionStats = {
val parsed = sm.ShardRegionStats.parseFrom(bytes) val parsed = sm.ShardRegionStats.parseFrom(bytes)
val stats: Map[String, Int] = parsed.getStatsList.asScala.map(e e.getKey -> e.getValue)(breakOut) val stats: Map[String, Int] = parsed.getStatsList.asScala.iterator.map(e e.getKey -> e.getValue).toMap
ShardRegionStats(stats) ShardRegionStats(stats)
} }

View file

@ -22,6 +22,7 @@ import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec import akka.remote.testkit.STMultiNodeSpec
import akka.remote.transport.ThrottlerTransportAdapter.Direction import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.testkit._ import akka.testkit._
import akka.util.ccompat.imm._
object ClusterShardingFailureSpec { object ClusterShardingFailureSpec {
case class Get(id: String) case class Get(id: String)
@ -125,8 +126,8 @@ abstract class ClusterShardingFailureSpec(config: ClusterShardingFailureSpecConf
within(remaining) { within(remaining) {
awaitAssert { awaitAssert {
cluster.state.members.map(_.uniqueAddress) should contain(cluster.selfUniqueAddress) cluster.state.members.unsorted.map(_.uniqueAddress) should contain(cluster.selfUniqueAddress)
cluster.state.members.map(_.status) should ===(Set(MemberStatus.Up)) cluster.state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
} }
} }
} }

View file

@ -19,6 +19,7 @@ import org.apache.commons.io.FileUtils
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.cluster.sharding.ShardRegion.GetClusterShardingStats import akka.cluster.sharding.ShardRegion.GetClusterShardingStats
import akka.cluster.sharding.ShardRegion.ClusterShardingStats import akka.cluster.sharding.ShardRegion.ClusterShardingStats
import akka.util.ccompat.imm._
object ClusterShardingMinMembersSpec { object ClusterShardingMinMembersSpec {
case object StopEntity case object StopEntity
@ -154,7 +155,7 @@ abstract class ClusterShardingMinMembersSpec(config: ClusterShardingMinMembersSp
within(remaining) { within(remaining) {
awaitAssert { awaitAssert {
cluster.state.members.size should ===(3) cluster.state.members.size should ===(3)
cluster.state.members.map(_.status) should ===(Set(MemberStatus.Up)) cluster.state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
} }
} }
enterBarrier("all-up") enterBarrier("all-up")

View file

@ -15,6 +15,7 @@ import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec }
import akka.testkit._ import akka.testkit._
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import org.apache.commons.io.FileUtils import org.apache.commons.io.FileUtils
import akka.util.ccompat.imm._
import scala.concurrent.duration._ import scala.concurrent.duration._
@ -175,7 +176,7 @@ abstract class ClusterShardingRememberEntitiesSpec(config: ClusterShardingRememb
within(remaining) { within(remaining) {
awaitAssert { awaitAssert {
cluster.state.members.size should ===(2) cluster.state.members.size should ===(2)
cluster.state.members.map(_.status) should ===(Set(MemberStatus.Up)) cluster.state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
} }
} }
} }

View file

@ -18,6 +18,7 @@ import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec import akka.remote.testkit.STMultiNodeSpec
import akka.testkit._ import akka.testkit._
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import akka.util.ccompat.imm._
object MultiDcClusterShardingSpec { object MultiDcClusterShardingSpec {
sealed trait EntityMsg { sealed trait EntityMsg {
@ -131,7 +132,7 @@ abstract class MultiDcClusterShardingSpec extends MultiNodeSpec(MultiDcClusterSh
awaitAssert({ awaitAssert({
withClue(s"Members: ${Cluster(system).state}") { withClue(s"Members: ${Cluster(system).state}") {
Cluster(system).state.members.size should ===(4) Cluster(system).state.members.size should ===(4)
Cluster(system).state.members.map(_.status) should ===(Set(MemberStatus.Up)) Cluster(system).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
} }
}, 10.seconds) }, 10.seconds)

View file

@ -16,6 +16,7 @@ import akka.testkit.AkkaSpec
import akka.testkit.TestActors.EchoActor import akka.testkit.TestActors.EchoActor
import akka.testkit.TestProbe import akka.testkit.TestProbe
import akka.testkit.WithLogCapturing import akka.testkit.WithLogCapturing
import akka.util.ccompat.imm._
object CoordinatedShutdownShardingSpec { object CoordinatedShutdownShardingSpec {
val config = val config =
@ -95,9 +96,9 @@ class CoordinatedShutdownShardingSpec extends AkkaSpec(CoordinatedShutdownShardi
within(10.seconds) { within(10.seconds) {
awaitAssert { awaitAssert {
Cluster(sys1).state.members.size should ===(2) Cluster(sys1).state.members.size should ===(2)
Cluster(sys1).state.members.map(_.status) should ===(Set(MemberStatus.Up)) Cluster(sys1).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
Cluster(sys2).state.members.size should ===(2) Cluster(sys2).state.members.size should ===(2)
Cluster(sys2).state.members.map(_.status) should ===(Set(MemberStatus.Up)) Cluster(sys2).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
} }
} }
@ -105,11 +106,11 @@ class CoordinatedShutdownShardingSpec extends AkkaSpec(CoordinatedShutdownShardi
within(10.seconds) { within(10.seconds) {
awaitAssert { awaitAssert {
Cluster(sys1).state.members.size should ===(3) Cluster(sys1).state.members.size should ===(3)
Cluster(sys1).state.members.map(_.status) should ===(Set(MemberStatus.Up)) Cluster(sys1).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
Cluster(sys2).state.members.size should ===(3) Cluster(sys2).state.members.size should ===(3)
Cluster(sys2).state.members.map(_.status) should ===(Set(MemberStatus.Up)) Cluster(sys2).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
Cluster(sys3).state.members.size should ===(3) Cluster(sys3).state.members.size should ===(3)
Cluster(sys3).state.members.map(_.status) should ===(Set(MemberStatus.Up)) Cluster(sys3).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
} }
} }

View file

@ -38,6 +38,7 @@ import com.typesafe.config.Config
import akka.remote.DeadlineFailureDetector import akka.remote.DeadlineFailureDetector
import akka.dispatch.Dispatchers import akka.dispatch.Dispatchers
import akka.util.MessageBuffer import akka.util.MessageBuffer
import akka.util.ccompat._
import scala.collection.immutable.{ HashMap, HashSet } import scala.collection.immutable.{ HashMap, HashSet }
object ClusterClientSettings { object ClusterClientSettings {
@ -332,7 +333,7 @@ final class ClusterClient(settings: ClusterClientSettings) extends Actor with Ac
val failureDetector = new DeadlineFailureDetector(acceptableHeartbeatPause, heartbeatInterval) val failureDetector = new DeadlineFailureDetector(acceptableHeartbeatPause, heartbeatInterval)
var contactPaths: HashSet[ActorPath] = var contactPaths: HashSet[ActorPath] =
initialContacts.to[HashSet] initialContacts.to(HashSet)
val initialContactsSel = val initialContactsSel =
contactPaths.map(context.actorSelection) contactPaths.map(context.actorSelection)
var contacts = initialContactsSel var contacts = initialContactsSel
@ -373,7 +374,7 @@ final class ClusterClient(settings: ClusterClientSettings) extends Actor with Ac
{ {
case Contacts(contactPoints) case Contacts(contactPoints)
if (contactPoints.nonEmpty) { if (contactPoints.nonEmpty) {
contactPaths = contactPoints.map(ActorPath.fromString).to[HashSet] contactPaths = contactPoints.map(ActorPath.fromString).to(HashSet)
contacts = contactPaths.map(context.actorSelection) contacts = contactPaths.map(context.actorSelection)
contacts foreach { _ ! Identify(Array.emptyByteArray) } contacts foreach { _ ! Identify(Array.emptyByteArray) }
} }
@ -423,7 +424,7 @@ final class ClusterClient(settings: ClusterClientSettings) extends Actor with Ac
case Contacts(contactPoints) case Contacts(contactPoints)
// refresh of contacts // refresh of contacts
if (contactPoints.nonEmpty) { if (contactPoints.nonEmpty) {
contactPaths = contactPoints.map(ActorPath.fromString).to[HashSet] contactPaths = contactPoints.map(ActorPath.fromString).to(HashSet)
contacts = contactPaths.map(context.actorSelection) contacts = contactPaths.map(context.actorSelection)
} }
publishContactPoints() publishContactPoints()
@ -951,7 +952,7 @@ final class ClusterReceptionist(pubSubMediator: ActorRef, settings: ClusterRecep
// is the same from all nodes (most of the time) and it also // is the same from all nodes (most of the time) and it also
// load balances the client connections among the nodes in the cluster. // load balances the client connections among the nodes in the cluster.
if (numberOfContacts >= nodes.size) { if (numberOfContacts >= nodes.size) {
val contacts = Contacts(nodes.map(a self.path.toStringWithAddress(a))(collection.breakOut)) val contacts = Contacts(nodes.iterator.map(a self.path.toStringWithAddress(a)).to(immutable.IndexedSeq))
if (log.isDebugEnabled) if (log.isDebugEnabled)
log.debug("Client [{}] gets contactPoints [{}] (all nodes)", sender().path, contacts.contactPoints.mkString(",")) log.debug("Client [{}] gets contactPoints [{}] (all nodes)", sender().path, contacts.contactPoints.mkString(","))
sender() ! contacts sender() ! contacts
@ -960,11 +961,11 @@ final class ClusterReceptionist(pubSubMediator: ActorRef, settings: ClusterRecep
// toStringWithAddress will use the remote address of the client // toStringWithAddress will use the remote address of the client
val a = consistentHash.nodeFor(sender().path.toStringWithAddress(cluster.selfAddress)) val a = consistentHash.nodeFor(sender().path.toStringWithAddress(cluster.selfAddress))
val slice = { val slice = {
val first = nodes.from(a).tail.take(numberOfContacts) val first = nodes.rangeFrom(a).tail.take(numberOfContacts)
if (first.size == numberOfContacts) first if (first.size == numberOfContacts) first
else first union nodes.take(numberOfContacts - first.size) else first union nodes.take(numberOfContacts - first.size)
} }
val contacts = Contacts(slice.map(a self.path.toStringWithAddress(a))(collection.breakOut)) val contacts = Contacts(slice.iterator.map(a self.path.toStringWithAddress(a)).to(immutable.IndexedSeq))
if (log.isDebugEnabled) if (log.isDebugEnabled)
log.debug("Client [{}] gets contactPoints [{}]", sender().path, contacts.contactPoints.mkString(",")) log.debug("Client [{}] gets contactPoints [{}]", sender().path, contacts.contactPoints.mkString(","))
sender() ! contacts sender() ! contacts
@ -992,7 +993,7 @@ final class ClusterReceptionist(pubSubMediator: ActorRef, settings: ClusterRecep
case SubscribeClusterClients case SubscribeClusterClients
val subscriber = sender() val subscriber = sender()
subscriber ! ClusterClients(clientInteractions.keySet.to[HashSet]) subscriber ! ClusterClients(clientInteractions.keySet.to(HashSet))
subscribers :+= subscriber subscribers :+= subscriber
context.watch(subscriber) context.watch(subscriber)
@ -1004,7 +1005,7 @@ final class ClusterReceptionist(pubSubMediator: ActorRef, settings: ClusterRecep
self.tell(UnsubscribeClusterClients, subscriber) self.tell(UnsubscribeClusterClients, subscriber)
case GetClusterClients case GetClusterClients
sender() ! ClusterClients(clientInteractions.keySet.to[HashSet]) sender() ! ClusterClients(clientInteractions.keySet.to(HashSet))
case CheckDeadlines case CheckDeadlines
clientInteractions = clientInteractions.filter { clientInteractions = clientInteractions.filter {
@ -1025,11 +1026,11 @@ final class ClusterReceptionist(pubSubMediator: ActorRef, settings: ClusterRecep
log.debug("Received new contact from [{}]", client.path) log.debug("Received new contact from [{}]", client.path)
val clusterClientUp = ClusterClientUp(client) val clusterClientUp = ClusterClientUp(client)
subscribers.foreach(_ ! clusterClientUp) subscribers.foreach(_ ! clusterClientUp)
clientsPublished = clientInteractions.keySet.to[HashSet] clientsPublished = clientInteractions.keySet.to(HashSet)
} }
def publishClientsUnreachable(): Unit = { def publishClientsUnreachable(): Unit = {
val publishableClients = clientInteractions.keySet.to[HashSet] val publishableClients = clientInteractions.keySet.to(HashSet)
for (c clientsPublished if !publishableClients.contains(c)) { for (c clientsPublished if !publishableClients.contains(c)) {
log.debug("Lost contact with [{}]", c.path) log.debug("Lost contact with [{}]", c.path)
val clusterClientUnreachable = ClusterClientUnreachable(c) val clusterClientUnreachable = ClusterClientUnreachable(c)

View file

@ -569,8 +569,8 @@ class DistributedPubSubMediator(settings: DistributedPubSubSettings) extends Act
case _ case _
(for { (for {
(_, bucket) registry (_, bucket) registry
valueHolder bucket.content.get(path) valueHolder bucket.content.get(path).toSeq
routee valueHolder.routee routee valueHolder.routee.toSeq
} yield routee).toVector } yield routee).toVector
} }
@ -751,8 +751,8 @@ class DistributedPubSubMediator(settings: DistributedPubSubSettings) extends Act
val refs = for { val refs = for {
(address, bucket) registry (address, bucket) registry
if !(allButSelf && address == selfAddress) // if we should skip sender() node and current address == self address => skip if !(allButSelf && address == selfAddress) // if we should skip sender() node and current address == self address => skip
valueHolder bucket.content.get(path) valueHolder bucket.content.get(path).toSeq
ref valueHolder.ref ref valueHolder.ref.toSeq
} yield ref } yield ref
if (refs.isEmpty) ignoreOrSendToDeadLetters(msg) if (refs.isEmpty) ignoreOrSendToDeadLetters(msg)
else refs.foreach(_.forward(msg)) else refs.foreach(_.forward(msg))
@ -793,7 +793,7 @@ class DistributedPubSubMediator(settings: DistributedPubSubSettings) extends Act
val topicPrefix = self.path.toStringWithoutAddress val topicPrefix = self.path.toStringWithoutAddress
(for { (for {
(_, bucket) registry (_, bucket) registry
(key, value) bucket.content (key, value) bucket.content.toSeq
if key.startsWith(topicPrefix) if key.startsWith(topicPrefix)
topic = key.substring(topicPrefix.length + 1) topic = key.substring(topicPrefix.length + 1)
if !topic.contains('/') // exclude group topics if !topic.contains('/') // exclude group topics

View file

@ -5,7 +5,6 @@
package akka.cluster.pubsub.protobuf package akka.cluster.pubsub.protobuf
import akka.serialization._ import akka.serialization._
import scala.collection.breakOut
import akka.actor.{ Address, ExtendedActorSystem } import akka.actor.{ Address, ExtendedActorSystem }
import java.io.{ ByteArrayInputStream, ByteArrayOutputStream } import java.io.{ ByteArrayInputStream, ByteArrayOutputStream }
import akka.protobuf.{ ByteString, MessageLite } import akka.protobuf.{ ByteString, MessageLite }
@ -17,6 +16,7 @@ import scala.collection.JavaConverters._
import akka.cluster.pubsub.DistributedPubSubMediator._ import akka.cluster.pubsub.DistributedPubSubMediator._
import akka.cluster.pubsub.DistributedPubSubMediator.Internal._ import akka.cluster.pubsub.DistributedPubSubMediator.Internal._
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.util.ccompat._
import scala.collection.immutable.TreeMap import scala.collection.immutable.TreeMap
import java.io.NotSerializableException import java.io.NotSerializableException
@ -127,8 +127,8 @@ private[akka] class DistributedPubSubMessageSerializer(val system: ExtendedActor
private def statusFromProto(status: dm.Status): Status = { private def statusFromProto(status: dm.Status): Status = {
val isReplyToStatus = if (status.hasReplyToStatus) status.getReplyToStatus else false val isReplyToStatus = if (status.hasReplyToStatus) status.getReplyToStatus else false
Status(status.getVersionsList.asScala.map(v Status(status.getVersionsList.asScala.iterator.map(v
addressFromProto(v.getAddress) v.getTimestamp)(breakOut), isReplyToStatus) addressFromProto(v.getAddress) v.getTimestamp).toMap, isReplyToStatus)
} }
private def deltaToProto(delta: Delta): dm.Delta = { private def deltaToProto(delta: Delta): dm.Delta = {
@ -154,9 +154,9 @@ private[akka] class DistributedPubSubMessageSerializer(val system: ExtendedActor
private def deltaFromProto(delta: dm.Delta): Delta = private def deltaFromProto(delta: dm.Delta): Delta =
Delta(delta.getBucketsList.asScala.toVector.map { b Delta(delta.getBucketsList.asScala.toVector.map { b
val content: TreeMap[String, ValueHolder] = b.getContentList.asScala.map { entry val content: TreeMap[String, ValueHolder] = scala.collection.immutable.TreeMap.from(b.getContentList.asScala.iterator.map { entry
entry.getKey ValueHolder(entry.getVersion, if (entry.hasRef) Some(resolveActorRef(entry.getRef)) else None) entry.getKey ValueHolder(entry.getVersion, if (entry.hasRef) Some(resolveActorRef(entry.getRef)) else None)
}(breakOut) })
Bucket(addressFromProto(b.getOwner), b.getVersion, content) Bucket(addressFromProto(b.getOwner), b.getVersion, content)
}) })

View file

@ -70,6 +70,7 @@ class ClusterSingletonLeavingSpeedSpec extends AkkaSpec("""
override def expectedTestDuration: FiniteDuration = 10.minutes override def expectedTestDuration: FiniteDuration = 10.minutes
def join(from: ActorSystem, to: ActorSystem, probe: ActorRef): Unit = { def join(from: ActorSystem, to: ActorSystem, probe: ActorRef): Unit = {
from.actorOf( from.actorOf(
ClusterSingletonManager.props( ClusterSingletonManager.props(
singletonProps = TheSingleton.props(probe), singletonProps = TheSingleton.props(probe),
@ -79,9 +80,10 @@ class ClusterSingletonLeavingSpeedSpec extends AkkaSpec("""
Cluster(from).join(Cluster(to).selfAddress) Cluster(from).join(Cluster(to).selfAddress)
within(15.seconds) { within(15.seconds) {
import akka.util.ccompat.imm._
awaitAssert { awaitAssert {
Cluster(from).state.members.map(_.uniqueAddress) should contain(Cluster(from).selfUniqueAddress) Cluster(from).state.members.map(_.uniqueAddress) should contain(Cluster(from).selfUniqueAddress)
Cluster(from).state.members.map(_.status) should ===(Set(MemberStatus.Up)) Cluster(from).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
} }
} }
} }

View file

@ -62,10 +62,11 @@ class ClusterSingletonRestart2Spec extends AkkaSpec("""
name = "echo") name = "echo")
within(45.seconds) { within(45.seconds) {
import akka.util.ccompat.imm._
awaitAssert { awaitAssert {
Cluster(from) join Cluster(to).selfAddress Cluster(from) join Cluster(to).selfAddress
Cluster(from).state.members.map(_.uniqueAddress) should contain(Cluster(from).selfUniqueAddress) Cluster(from).state.members.map(_.uniqueAddress) should contain(Cluster(from).selfUniqueAddress)
Cluster(from).state.members.map(_.status) should ===(Set(MemberStatus.Up)) Cluster(from).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
} }
} }
} }

View file

@ -43,10 +43,11 @@ class ClusterSingletonRestartSpec extends AkkaSpec("""
name = "echo") name = "echo")
within(10.seconds) { within(10.seconds) {
import akka.util.ccompat.imm._
awaitAssert { awaitAssert {
Cluster(from) join Cluster(to).selfAddress Cluster(from) join Cluster(to).selfAddress
Cluster(from).state.members.map(_.uniqueAddress) should contain(Cluster(from).selfUniqueAddress) Cluster(from).state.members.map(_.uniqueAddress) should contain(Cluster(from).selfUniqueAddress)
Cluster(from).state.members.map(_.status) should ===(Set(MemberStatus.Up)) Cluster(from).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
} }
} }
} }

View file

@ -16,7 +16,7 @@ import akka.dispatch.MonitorableThreadFactory
import akka.event.{ Logging, LoggingAdapter } import akka.event.{ Logging, LoggingAdapter }
import akka.japi.Util import akka.japi.Util
import akka.pattern._ import akka.pattern._
import akka.remote.{ DefaultFailureDetectorRegistry, _ } import akka.remote.{ UniqueAddress _, _ }
import com.typesafe.config.{ Config, ConfigFactory } import com.typesafe.config.{ Config, ConfigFactory }
import scala.annotation.varargs import scala.annotation.varargs

View file

@ -1258,9 +1258,7 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef, joinConfigCompatCh
} }
if (changedMembers.nonEmpty) { if (changedMembers.nonEmpty) {
// replace changed members val newGossip = localGossip.update(changedMembers)
val newMembers = changedMembers union localMembers
val newGossip = localGossip.copy(members = newMembers)
updateLatestGossip(newGossip) updateLatestGossip(newGossip)
// log status changes // log status changes

View file

@ -15,8 +15,8 @@ import akka.event.EventStream
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics } import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
import akka.actor.DeadLetterSuppression import akka.actor.DeadLetterSuppression
import akka.annotation.{ DoNotInherit, InternalApi } import akka.annotation.{ DoNotInherit, InternalApi }
import akka.util.ccompat._
import scala.collection.breakOut
import scala.runtime.AbstractFunction5 import scala.runtime.AbstractFunction5
/** /**
@ -162,7 +162,7 @@ object ClusterEvent {
/** /**
* All data centers in the cluster * All data centers in the cluster
*/ */
def allDataCenters: Set[String] = members.map(_.dataCenter)(breakOut) def allDataCenters: Set[String] = members.iterator.map(_.dataCenter).to(immutable.Set)
/** /**
* Java API: All data centers in the cluster * Java API: All data centers in the cluster
@ -380,10 +380,10 @@ object ClusterEvent {
else { else {
val newGossip = newState.latestGossip val newGossip = newState.latestGossip
val oldUnreachableNodes = oldState.dcReachabilityNoOutsideNodes.allUnreachableOrTerminated val oldUnreachableNodes = oldState.dcReachabilityNoOutsideNodes.allUnreachableOrTerminated
newState.dcReachabilityNoOutsideNodes.allUnreachableOrTerminated.collect { newState.dcReachabilityNoOutsideNodes.allUnreachableOrTerminated.iterator.collect {
case node if !oldUnreachableNodes.contains(node) && node != newState.selfUniqueAddress case node if !oldUnreachableNodes.contains(node) && node != newState.selfUniqueAddress
UnreachableMember(newGossip.member(node)) UnreachableMember(newGossip.member(node))
}(collection.breakOut) }.to(immutable.IndexedSeq)
} }
/** /**
@ -393,10 +393,10 @@ object ClusterEvent {
if (newState eq oldState) Nil if (newState eq oldState) Nil
else { else {
val newGossip = newState.latestGossip val newGossip = newState.latestGossip
oldState.dcReachabilityNoOutsideNodes.allUnreachable.collect { oldState.dcReachabilityNoOutsideNodes.allUnreachable.iterator.collect {
case node if newGossip.hasMember(node) && newState.dcReachabilityNoOutsideNodes.isReachable(node) && node != newState.selfUniqueAddress case node if newGossip.hasMember(node) && newState.dcReachabilityNoOutsideNodes.isReachable(node) && node != newState.selfUniqueAddress
ReachableMember(newGossip.member(node)) ReachableMember(newGossip.member(node))
}(collection.breakOut) }.to(immutable.IndexedSeq)
} }
/** /**
@ -418,7 +418,7 @@ object ClusterEvent {
if (newState eq oldState) Nil if (newState eq oldState) Nil
else { else {
val otherDcs = (oldState.latestGossip.allDataCenters union newState.latestGossip.allDataCenters) - newState.selfDc val otherDcs = (oldState.latestGossip.allDataCenters union newState.latestGossip.allDataCenters) - newState.selfDc
otherDcs.filterNot(isReachable(newState, oldState.dcReachability.allUnreachableOrTerminated)).map(UnreachableDataCenter)(collection.breakOut) otherDcs.filterNot(isReachable(newState, oldState.dcReachability.allUnreachableOrTerminated)).iterator.map(UnreachableDataCenter).to(immutable.IndexedSeq)
} }
} }
@ -433,7 +433,7 @@ object ClusterEvent {
val oldUnreachableDcs = otherDcs.filterNot(isReachable(oldState, Set())) val oldUnreachableDcs = otherDcs.filterNot(isReachable(oldState, Set()))
val currentUnreachableDcs = otherDcs.filterNot(isReachable(newState, Set())) val currentUnreachableDcs = otherDcs.filterNot(isReachable(newState, Set()))
(oldUnreachableDcs diff currentUnreachableDcs).map(ReachableDataCenter)(collection.breakOut) (oldUnreachableDcs diff currentUnreachableDcs).iterator.map(ReachableDataCenter).to(immutable.IndexedSeq)
} }
} }
@ -451,7 +451,8 @@ object ClusterEvent {
case (_, newMember :: oldMember :: Nil) if newMember.status != oldMember.status || newMember.upNumber != oldMember.upNumber case (_, newMember :: oldMember :: Nil) if newMember.status != oldMember.status || newMember.upNumber != oldMember.upNumber
newMember newMember
} }
val memberEvents = (newMembers ++ changedMembers) collect { import akka.util.ccompat.imm._
val memberEvents = (newMembers ++ changedMembers).unsorted collect {
case m if m.status == Joining MemberJoined(m) case m if m.status == Joining MemberJoined(m)
case m if m.status == WeaklyUp MemberWeaklyUp(m) case m if m.status == WeaklyUp MemberWeaklyUp(m)
case m if m.status == Up MemberUp(m) case m if m.status == Up MemberUp(m)
@ -462,7 +463,7 @@ object ClusterEvent {
} }
val removedMembers = oldGossip.members diff newGossip.members val removedMembers = oldGossip.members diff newGossip.members
val removedEvents = removedMembers.map(m MemberRemoved(m.copy(status = Removed), m.status)) val removedEvents = removedMembers.unsorted.map(m MemberRemoved(m.copy(status = Removed), m.status))
(new VectorBuilder[MemberEvent]() ++= removedEvents ++= memberEvents).result() (new VectorBuilder[MemberEvent]() ++= removedEvents ++= memberEvents).result()
} }
@ -572,8 +573,8 @@ private[cluster] final class ClusterDomainEventPublisher extends Actor with Acto
unreachable = unreachable, unreachable = unreachable,
seenBy = membershipState.latestGossip.seenBy.map(_.address), seenBy = membershipState.latestGossip.seenBy.map(_.address),
leader = membershipState.leader.map(_.address), leader = membershipState.leader.map(_.address),
roleLeaderMap = membershipState.latestGossip.allRoles.map(r roleLeaderMap = membershipState.latestGossip.allRoles.iterator.map(r
r membershipState.roleLeader(r).map(_.address))(collection.breakOut), r membershipState.roleLeader(r).map(_.address)).toMap,
unreachableDataCenters) unreachableDataCenters)
receiver ! state receiver ! state
} }

View file

@ -14,6 +14,7 @@ import akka.cluster.ClusterEvent._
import akka.remote.FailureDetectorRegistry import akka.remote.FailureDetectorRegistry
import akka.remote.HeartbeatMessage import akka.remote.HeartbeatMessage
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.util.ccompat._
/** /**
* INTERNAL API. * INTERNAL API.
@ -338,13 +339,13 @@ private[cluster] final case class HeartbeatNodeRing(
take(n - 1, iter, acc + next) // include the reachable take(n - 1, iter, acc + next) // include the reachable
} }
val (remaining, slice1) = take(monitoredByNrOfMembers, nodeRing.from(sender).tail.iterator, Set.empty) val (remaining, slice1) = take(monitoredByNrOfMembers, nodeRing.rangeFrom(sender).tail.iterator, Set.empty)
val slice = val slice =
if (remaining == 0) if (remaining == 0)
slice1 slice1
else { else {
// wrap around // wrap around
val (_, slice2) = take(remaining, nodeRing.to(sender).iterator.filterNot(_ == sender), slice1) val (_, slice2) = take(remaining, nodeRing.rangeTo(sender).iterator.filterNot(_ == sender), slice1)
slice2 slice2
} }

View file

@ -10,8 +10,10 @@ import akka.cluster.ClusterEvent._
import akka.cluster.ClusterSettings.DataCenter import akka.cluster.ClusterSettings.DataCenter
import akka.remote.FailureDetectorRegistry import akka.remote.FailureDetectorRegistry
import akka.util.ConstantFun import akka.util.ConstantFun
import akka.util.ccompat._
import scala.collection.{ SortedSet, breakOut } import scala.collection.SortedSet
import scala.collection.immutable
/** /**
* INTERNAL API * INTERNAL API
@ -270,8 +272,8 @@ private[cluster] final case class CrossDcHeartbeatingState(
val allOtherNodes = otherDcs.values val allOtherNodes = otherDcs.values
allOtherNodes.flatMap( allOtherNodes.flatMap(
_.take(nrOfMonitoredNodesPerDc) _.take(nrOfMonitoredNodesPerDc).iterator
.map(_.uniqueAddress)(breakOut)).toSet .map(_.uniqueAddress).to(immutable.IndexedSeq)).toSet
} }
/** Lists addresses in diven DataCenter that this node should send heartbeats to */ /** Lists addresses in diven DataCenter that this node should send heartbeats to */
@ -280,8 +282,8 @@ private[cluster] final case class CrossDcHeartbeatingState(
else { else {
val otherNodes = state.getOrElse(dc, emptyMembersSortedSet) val otherNodes = state.getOrElse(dc, emptyMembersSortedSet)
otherNodes otherNodes
.take(nrOfMonitoredNodesPerDc) .take(nrOfMonitoredNodesPerDc).iterator
.map(_.uniqueAddress)(breakOut) .map(_.uniqueAddress).to(immutable.Set)
} }
def allMembers: Iterable[Member] = def allMembers: Iterable[Member] =

View file

@ -87,7 +87,7 @@ private[cluster] final case class Gossip(
} }
@transient private lazy val membersMap: Map[UniqueAddress, Member] = @transient private lazy val membersMap: Map[UniqueAddress, Member] =
members.map(m m.uniqueAddress m)(collection.breakOut) members.iterator.map(m m.uniqueAddress m).toMap
@transient lazy val isMultiDc = @transient lazy val isMultiDc =
if (members.size <= 1) false if (members.size <= 1) false
@ -209,7 +209,7 @@ private[cluster] final case class Gossip(
} }
def update(updatedMembers: immutable.SortedSet[Member]): Gossip = { def update(updatedMembers: immutable.SortedSet[Member]): Gossip = {
copy(members = updatedMembers union members) copy(members = updatedMembers union (members diff updatedMembers))
} }
/** /**

View file

@ -8,6 +8,7 @@ import java.util
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
import akka.annotation.{ DoNotInherit, InternalApi } import akka.annotation.{ DoNotInherit, InternalApi }
import akka.util.ccompat._
import com.typesafe.config.{ Config, ConfigFactory, ConfigValue } import com.typesafe.config.{ Config, ConfigFactory, ConfigValue }
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
@ -48,7 +49,7 @@ object JoinConfigCompatChecker {
} }
if (result.isEmpty) Valid if (result.isEmpty) Valid
else Invalid(result.to[im.Seq]) else Invalid(result.to(im.Seq))
} }
/** /**
@ -78,7 +79,7 @@ object JoinConfigCompatChecker {
} }
if (incompatibleKeys.isEmpty) Valid if (incompatibleKeys.isEmpty) Valid
else Invalid(incompatibleKeys.to[im.Seq]) else Invalid(incompatibleKeys.to(im.Seq))
} }
exists(requiredKeys, toCheck) ++ checkEquality exists(requiredKeys, toCheck) ++ checkEquality
@ -123,7 +124,7 @@ object JoinConfigCompatChecker {
*/ */
@InternalApi @InternalApi
private[cluster] def removeSensitiveKeys(config: Config, clusterSettings: ClusterSettings): im.Seq[String] = { private[cluster] def removeSensitiveKeys(config: Config, clusterSettings: ClusterSettings): im.Seq[String] = {
val existingKeys = config.entrySet().asScala.map(_.getKey).to[im.Seq] val existingKeys = config.entrySet().asScala.map(_.getKey).to(im.Seq)
removeSensitiveKeys(existingKeys, clusterSettings) removeSensitiveKeys(existingKeys, clusterSettings)
} }
@ -147,7 +148,7 @@ object JoinConfigCompatChecker {
new JoinConfigCompatChecker { new JoinConfigCompatChecker {
override val requiredKeys: im.Seq[String] = { override val requiredKeys: im.Seq[String] = {
// Always include akka.version (used in join logging) // Always include akka.version (used in join logging)
"akka.version" +: checkers.flatMap(_.requiredKeys).to[im.Seq] "akka.version" +: checkers.flatMap(_.requiredKeys).to(im.Seq)
} }
override def check(toValidate: Config, clusterConfig: Config): ConfigValidation = override def check(toValidate: Config, clusterConfig: Config): ConfigValidation =
checkers.foldLeft(Valid: ConfigValidation) { (acc, checker) checkers.foldLeft(Valid: ConfigValidation) { (acc, checker)

View file

@ -11,9 +11,9 @@ import scala.collection.SortedSet
import akka.cluster.ClusterSettings.DataCenter import akka.cluster.ClusterSettings.DataCenter
import akka.cluster.MemberStatus._ import akka.cluster.MemberStatus._
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.util.ccompat._
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.collection.breakOut
import scala.util.Random import scala.util.Random
/** /**
@ -275,10 +275,10 @@ import scala.util.Random
if (preferNodesWithDifferentView(state)) { if (preferNodesWithDifferentView(state)) {
// If it's time to try to gossip to some nodes with a different view // If it's time to try to gossip to some nodes with a different view
// gossip to a random alive same dc member with preference to a member with older gossip version // gossip to a random alive same dc member with preference to a member with older gossip version
latestGossip.members.collect { latestGossip.members.iterator.collect {
case m if m.dataCenter == state.selfDc && !latestGossip.seenByNode(m.uniqueAddress) && state.validNodeForGossip(m.uniqueAddress) case m if m.dataCenter == state.selfDc && !latestGossip.seenByNode(m.uniqueAddress) && state.validNodeForGossip(m.uniqueAddress)
m.uniqueAddress m.uniqueAddress
}(breakOut) }.to(Vector)
} else Vector.empty } else Vector.empty
// Fall back to localGossip // Fall back to localGossip

View file

@ -7,7 +7,7 @@ package akka.cluster
import akka.annotation.InternalApi import akka.annotation.InternalApi
import scala.collection.immutable import scala.collection.immutable
import scala.collection.breakOut import akka.util.ccompat._
/** /**
* INTERNAL API * INTERNAL API
@ -72,9 +72,8 @@ private[cluster] class Reachability private (
(observerRowsMap, allUnreachable, allTerminated) (observerRowsMap, allUnreachable, allTerminated)
} else { } else {
val mapBuilder = scala.collection.mutable.Map.empty[UniqueAddress, Map[UniqueAddress, Reachability.Record]] val mapBuilder = scala.collection.mutable.Map.empty[UniqueAddress, Map[UniqueAddress, Reachability.Record]]
import scala.collection.mutable.SetBuilder var allTerminated = Set.empty[UniqueAddress]
val terminatedBuilder = new SetBuilder[UniqueAddress, Set[UniqueAddress]](Set.empty) var allUnreachable = Set.empty[UniqueAddress]
val unreachableBuilder = new SetBuilder[UniqueAddress, Set[UniqueAddress]](Set.empty)
records foreach { r records foreach { r
val m = mapBuilder.get(r.observer) match { val m = mapBuilder.get(r.observer) match {
@ -83,15 +82,13 @@ private[cluster] class Reachability private (
} }
mapBuilder += (r.observer m) mapBuilder += (r.observer m)
if (r.status == Unreachable) unreachableBuilder += r.subject if (r.status == Unreachable) allUnreachable += r.subject
else if (r.status == Terminated) terminatedBuilder += r.subject else if (r.status == Terminated) allTerminated += r.subject
} }
val observerRowsMap: Map[UniqueAddress, Map[UniqueAddress, Reachability.Record]] = mapBuilder.toMap val observerRowsMap: Map[UniqueAddress, Map[UniqueAddress, Reachability.Record]] = mapBuilder.toMap
val allTerminated: Set[UniqueAddress] = terminatedBuilder.result()
val allUnreachable: Set[UniqueAddress] = unreachableBuilder.result() diff allTerminated
(observerRowsMap, allUnreachable, allTerminated) (observerRowsMap, allUnreachable diff allTerminated, allTerminated)
} }
} }
@ -195,7 +192,7 @@ private[cluster] class Reachability private (
} }
def remove(nodes: Iterable[UniqueAddress]): Reachability = { def remove(nodes: Iterable[UniqueAddress]): Reachability = {
val nodesSet = nodes.to[immutable.HashSet] val nodesSet = nodes.to(immutable.HashSet)
val newRecords = records.filterNot(r nodesSet(r.observer) || nodesSet(r.subject)) val newRecords = records.filterNot(r nodesSet(r.observer) || nodesSet(r.subject))
val newVersions = versions -- nodes val newVersions = versions -- nodes
Reachability(newRecords, newVersions) Reachability(newRecords, newVersions)
@ -265,16 +262,16 @@ private[cluster] class Reachability private (
observerRows(observer) match { observerRows(observer) match {
case None Set.empty case None Set.empty
case Some(observerRows) case Some(observerRows)
observerRows.collect { observerRows.iterator.collect {
case (subject, record) if record.status == Unreachable subject case (subject, record) if record.status == Unreachable subject
}(breakOut) }.to(immutable.Set)
} }
def observersGroupedByUnreachable: Map[UniqueAddress, Set[UniqueAddress]] = { def observersGroupedByUnreachable: Map[UniqueAddress, Set[UniqueAddress]] = {
records.groupBy(_.subject).collect { records.groupBy(_.subject).collect {
case (subject, records) if records.exists(_.status == Unreachable) case (subject, records) if records.exists(_.status == Unreachable)
val observers: Set[UniqueAddress] = val observers: Set[UniqueAddress] =
records.collect { case r if r.status == Unreachable r.observer }(breakOut) records.iterator.collect { case r if r.status == Unreachable r.observer }.to(immutable.Set)
(subject observers) (subject observers)
} }
} }

View file

@ -21,6 +21,8 @@ import akka.annotation.InternalApi
import akka.cluster.InternalClusterAction._ import akka.cluster.InternalClusterAction._
import akka.cluster.routing.{ ClusterRouterPool, ClusterRouterPoolSettings } import akka.cluster.routing.{ ClusterRouterPool, ClusterRouterPoolSettings }
import akka.routing.Pool import akka.routing.Pool
import akka.util.ccompat._
import akka.util.ccompat.imm._
import com.typesafe.config.{ Config, ConfigFactory, ConfigRenderOptions } import com.typesafe.config.{ Config, ConfigFactory, ConfigRenderOptions }
/** /**
@ -370,9 +372,9 @@ final class ClusterMessageSerializer(val system: ExtendedActorSystem) extends Se
val allMembers = gossip.members.toVector val allMembers = gossip.members.toVector
val allAddresses: Vector[UniqueAddress] = allMembers.map(_.uniqueAddress) ++ gossip.tombstones.keys val allAddresses: Vector[UniqueAddress] = allMembers.map(_.uniqueAddress) ++ gossip.tombstones.keys
val addressMapping = allAddresses.zipWithIndex.toMap val addressMapping = allAddresses.zipWithIndex.toMap
val allRoles = allMembers.foldLeft(Set.empty[String])((acc, m) acc union m.roles).to[Vector] val allRoles = allMembers.foldLeft(Set.empty[String])((acc, m) acc union m.roles).to(Vector)
val roleMapping = allRoles.zipWithIndex.toMap val roleMapping = allRoles.zipWithIndex.toMap
val allHashes = gossip.version.versions.keys.to[Vector] val allHashes = gossip.version.versions.keys.to(Vector)
val hashMapping = allHashes.zipWithIndex.toMap val hashMapping = allHashes.zipWithIndex.toMap
def mapUniqueAddress(uniqueAddress: UniqueAddress): Integer = mapWithErrorMessage(addressMapping, uniqueAddress, "address") def mapUniqueAddress(uniqueAddress: UniqueAddress): Integer = mapWithErrorMessage(addressMapping, uniqueAddress, "address")
@ -403,7 +405,7 @@ final class ClusterMessageSerializer(val system: ExtendedActorSystem) extends Se
.build() .build()
val reachability = reachabilityToProto(gossip.overview.reachability) val reachability = reachabilityToProto(gossip.overview.reachability)
val members = (gossip.members: Set[Member] /* 2.13.0-M5 change cast to .unsorted */ ).map(memberToProto _) val members = gossip.members.unsorted.map(memberToProto _)
val seen = gossip.overview.seen.map(mapUniqueAddress) val seen = gossip.overview.seen.map(mapUniqueAddress)
val overview = cm.GossipOverview.newBuilder.addAllSeen(seen.asJava). val overview = cm.GossipOverview.newBuilder.addAllSeen(seen.asJava).
@ -447,11 +449,10 @@ final class ClusterMessageSerializer(val system: ExtendedActorSystem) extends Se
gossipStatusFromProto(cm.GossipStatus.parseFrom(bytes)) gossipStatusFromProto(cm.GossipStatus.parseFrom(bytes))
private def gossipFromProto(gossip: cm.Gossip): Gossip = { private def gossipFromProto(gossip: cm.Gossip): Gossip = {
import scala.collection.breakOut
val addressMapping: Vector[UniqueAddress] = val addressMapping: Vector[UniqueAddress] =
gossip.getAllAddressesList.asScala.map(uniqueAddressFromProto)(breakOut) gossip.getAllAddressesList.asScala.iterator.map(uniqueAddressFromProto).to(immutable.Vector)
val roleMapping: Vector[String] = gossip.getAllRolesList.asScala.map(identity)(breakOut) val roleMapping: Vector[String] = gossip.getAllRolesList.asScala.iterator.map(identity).to(immutable.Vector)
val hashMapping: Vector[String] = gossip.getAllHashesList.asScala.map(identity)(breakOut) val hashMapping: Vector[String] = gossip.getAllHashesList.asScala.iterator.map(identity).to(immutable.Vector)
def reachabilityFromProto(observerReachability: Iterable[cm.ObserverReachability]): Reachability = { def reachabilityFromProto(observerReachability: Iterable[cm.ObserverReachability]): Reachability = {
val recordBuilder = new immutable.VectorBuilder[Reachability.Record] val recordBuilder = new immutable.VectorBuilder[Reachability.Record]
@ -492,20 +493,19 @@ final class ClusterMessageSerializer(val system: ExtendedActorSystem) extends Se
def tombstoneFromProto(tombstone: cm.Tombstone): (UniqueAddress, Long) = def tombstoneFromProto(tombstone: cm.Tombstone): (UniqueAddress, Long) =
(addressMapping(tombstone.getAddressIndex), tombstone.getTimestamp) (addressMapping(tombstone.getAddressIndex), tombstone.getTimestamp)
val members: immutable.SortedSet[Member] = gossip.getMembersList.asScala.map(memberFromProto)(breakOut) val members: immutable.SortedSet[Member] = gossip.getMembersList.asScala.iterator.map(memberFromProto).to(immutable.SortedSet)
val reachability = reachabilityFromProto(gossip.getOverview.getObserverReachabilityList.asScala) val reachability = reachabilityFromProto(gossip.getOverview.getObserverReachabilityList.asScala)
val seen: Set[UniqueAddress] = gossip.getOverview.getSeenList.asScala.map(addressMapping(_))(breakOut) val seen: Set[UniqueAddress] = gossip.getOverview.getSeenList.asScala.iterator.map(addressMapping(_)).to(immutable.Set)
val overview = GossipOverview(seen, reachability) val overview = GossipOverview(seen, reachability)
val tombstones: Map[UniqueAddress, Long] = gossip.getTombstonesList.asScala.map(tombstoneFromProto)(breakOut) val tombstones: Map[UniqueAddress, Long] = gossip.getTombstonesList.asScala.iterator.map(tombstoneFromProto).toMap
Gossip(members, overview, vectorClockFromProto(gossip.getVersion, hashMapping), tombstones) Gossip(members, overview, vectorClockFromProto(gossip.getVersion, hashMapping), tombstones)
} }
private def vectorClockFromProto(version: cm.VectorClock, hashMapping: immutable.Seq[String]) = { private def vectorClockFromProto(version: cm.VectorClock, hashMapping: immutable.Seq[String]) = {
import scala.collection.breakOut VectorClock(scala.collection.immutable.TreeMap.from(version.getVersionsList.asScala.iterator.map(
VectorClock(version.getVersionsList.asScala.map( v (VectorClock.Node.fromHash(hashMapping(v.getHashIndex)), v.getTimestamp))))
v (VectorClock.Node.fromHash(hashMapping(v.getHashIndex)), v.getTimestamp))(breakOut))
} }
private def gossipEnvelopeFromProto(envelope: cm.GossipEnvelope): GossipEnvelope = { private def gossipEnvelopeFromProto(envelope: cm.GossipEnvelope): GossipEnvelope = {

View file

@ -354,11 +354,13 @@ private[akka] class ClusterRouterPoolActor(
None None
} else { } else {
// find the node with least routees // find the node with least routees
val numberOfRouteesPerNode: Map[Address, Int] = val numberOfRouteesPerNode: Map[Address, Int] = {
currentRoutees.foldLeft(currentNodes.map(_ 0).toMap.withDefaultValue(0)) { (acc, x) val nodeMap: Map[Address, Int] = currentNodes.map(_ 0).toMap.withDefaultValue(0)
currentRoutees.foldLeft(nodeMap) { (acc, x)
val address = fullAddress(x) val address = fullAddress(x)
acc + (address (acc(address) + 1)) acc + (address (acc(address) + 1))
} }
}
val (address, count) = numberOfRouteesPerNode.minBy(_._2) val (address, count) = numberOfRouteesPerNode.minBy(_._2)
if (count < settings.maxInstancesPerNode) Some(address) else None if (count < settings.maxInstancesPerNode) Some(address) else None

View file

@ -10,6 +10,7 @@ import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec import akka.remote.testkit.MultiNodeSpec
import akka.testkit._ import akka.testkit._
import akka.cluster.MemberStatus._ import akka.cluster.MemberStatus._
import akka.util.ccompat.imm._
object MinMembersBeforeUpMultiJvmSpec extends MultiNodeConfig { object MinMembersBeforeUpMultiJvmSpec extends MultiNodeConfig {
val first = role("first") val first = role("first")
@ -131,12 +132,12 @@ abstract class MinMembersBeforeUpBase(multiNodeConfig: MultiNodeConfig)
clusterView.refreshCurrentState() clusterView.refreshCurrentState()
clusterView.members.map(_.address) should ===(expectedAddresses) clusterView.members.map(_.address) should ===(expectedAddresses)
} }
clusterView.members.map(_.status) should ===(Set(Joining)) clusterView.members.unsorted.map(_.status) should ===(Set(Joining))
// and it should not change // and it should not change
1 to 5 foreach { _ 1 to 5 foreach { _
Thread.sleep(1000) Thread.sleep(1000)
clusterView.members.map(_.address) should ===(expectedAddresses) clusterView.members.map(_.address) should ===(expectedAddresses)
clusterView.members.map(_.status) should ===(Set(Joining)) clusterView.members.unsorted.map(_.status) should ===(Set(Joining))
} }
} }
enterBarrier("second-joined") enterBarrier("second-joined")

View file

@ -14,6 +14,7 @@ import com.typesafe.config.ConfigFactory
import scala.collection.immutable import scala.collection.immutable
import scala.collection.immutable.SortedSet import scala.collection.immutable.SortedSet
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.util.ccompat.imm._
object MultiDcHeartbeatTakingOverSpecMultiJvmSpec extends MultiNodeConfig { object MultiDcHeartbeatTakingOverSpecMultiJvmSpec extends MultiNodeConfig {
val first = role("first") // alpha val first = role("first") // alpha
@ -77,10 +78,10 @@ abstract class MultiDcHeartbeatTakingOverSpec extends MultiNodeSpec(MultiDcHeart
// these will be filled in during the initial phase of the test ----------- // these will be filled in during the initial phase of the test -----------
var expectedAlphaHeartbeaterNodes: SortedSet[Member] = SortedSet.empty var expectedAlphaHeartbeaterNodes: SortedSet[Member] = SortedSet.empty
var expectedAlphaHeartbeaterRoles: SortedSet[RoleName] = SortedSet.empty var expectedAlphaHeartbeaterRoles: List[RoleName] = List.empty
var expectedBetaHeartbeaterNodes: SortedSet[Member] = SortedSet.empty var expectedBetaHeartbeaterNodes: SortedSet[Member] = SortedSet.empty
var expectedBetaHeartbeaterRoles: SortedSet[RoleName] = SortedSet.empty var expectedBetaHeartbeaterRoles: List[RoleName] = List.empty
var expectedNoActiveHeartbeatSenderRoles: Set[RoleName] = Set.empty var expectedNoActiveHeartbeatSenderRoles: Set[RoleName] = Set.empty
// end of these will be filled in during the initial phase of the test ----------- // end of these will be filled in during the initial phase of the test -----------
@ -191,8 +192,8 @@ abstract class MultiDcHeartbeatTakingOverSpec extends MultiNodeSpec(MultiDcHeart
private[cluster] def takeNOldestMembers(dataCenter: ClusterSettings.DataCenter, n: Int): immutable.SortedSet[Member] = private[cluster] def takeNOldestMembers(dataCenter: ClusterSettings.DataCenter, n: Int): immutable.SortedSet[Member] =
membersByAge(dataCenter).take(n) membersByAge(dataCenter).take(n)
private def membersAsRoles(ms: SortedSet[Member]): SortedSet[RoleName] = { private def membersAsRoles(ms: SortedSet[Member]): List[RoleName] = {
val res = ms.flatMap(m roleName(m.address)) val res = ms.toList.flatMap(m roleName(m.address))
require(res.size == ms.size, s"Not all members were converted to roles! Got: ${ms}, found ${res}") require(res.size == ms.size, s"Not all members were converted to roles! Got: ${ms}, found ${res}")
res res
} }

View file

@ -17,6 +17,7 @@ import akka.testkit._
import akka.testkit.TestEvent._ import akka.testkit.TestEvent._
import akka.actor.{ Actor, ActorRef, ActorSystem, Address, Deploy, PoisonPill, Props, RootActorPath } import akka.actor.{ Actor, ActorRef, ActorSystem, Address, Deploy, PoisonPill, Props, RootActorPath }
import akka.event.Logging.ErrorLevel import akka.event.Logging.ErrorLevel
import akka.util.ccompat._
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.collection.immutable import scala.collection.immutable
@ -24,6 +25,7 @@ import java.util.concurrent.ConcurrentHashMap
import akka.remote.DefaultFailureDetectorRegistry import akka.remote.DefaultFailureDetectorRegistry
import akka.cluster.ClusterEvent.{ MemberEvent, MemberRemoved } import akka.cluster.ClusterEvent.{ MemberEvent, MemberRemoved }
import akka.util.ccompat.imm._
import scala.concurrent.Await import scala.concurrent.Await
@ -273,7 +275,7 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec with WatchedByCoro
* be determined from the `RoleName`. * be determined from the `RoleName`.
*/ */
def assertLeader(nodesInCluster: RoleName*): Unit = def assertLeader(nodesInCluster: RoleName*): Unit =
if (nodesInCluster.contains(myself)) assertLeaderIn(nodesInCluster.to[immutable.Seq]) if (nodesInCluster.contains(myself)) assertLeaderIn(nodesInCluster.to(immutable.Seq))
/** /**
* Assert that the cluster has elected the correct leader * Assert that the cluster has elected the correct leader
@ -310,7 +312,7 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec with WatchedByCoro
if (!canNotBePartOfMemberRing.isEmpty) // don't run this on an empty set if (!canNotBePartOfMemberRing.isEmpty) // don't run this on an empty set
awaitAssert(canNotBePartOfMemberRing foreach (a clusterView.members.map(_.address) should not contain (a))) awaitAssert(canNotBePartOfMemberRing foreach (a clusterView.members.map(_.address) should not contain (a)))
awaitAssert(clusterView.members.size should ===(numberOfMembers)) awaitAssert(clusterView.members.size should ===(numberOfMembers))
awaitAssert(clusterView.members.map(_.status) should ===(Set(MemberStatus.Up))) awaitAssert(clusterView.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up)))
// clusterView.leader is updated by LeaderChanged, await that to be updated also // clusterView.leader is updated by LeaderChanged, await that to be updated also
val expectedLeader = clusterView.members.collectFirst { val expectedLeader = clusterView.members.collectFirst {
case m if m.dataCenter == cluster.settings.SelfDataCenter m.address case m if m.dataCenter == cluster.settings.SelfDataCenter m.address

View file

@ -7,6 +7,7 @@ package akka.cluster
import akka.remote.testkit.MultiNodeConfig import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec import akka.remote.testkit.MultiNodeSpec
import akka.testkit._ import akka.testkit._
import akka.util.ccompat.imm._
object NodeMembershipMultiJvmSpec extends MultiNodeConfig { object NodeMembershipMultiJvmSpec extends MultiNodeConfig {
val first = role("first") val first = role("first")
@ -40,7 +41,7 @@ abstract class NodeMembershipSpec
cluster.join(first) cluster.join(first)
awaitAssert(clusterView.members.size should ===(2)) awaitAssert(clusterView.members.size should ===(2))
assertMembers(clusterView.members, first, second) assertMembers(clusterView.members, first, second)
awaitAssert(clusterView.members.map(_.status) should ===(Set(MemberStatus.Up))) awaitAssert(clusterView.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up)))
} }
enterBarrier("after-1") enterBarrier("after-1")
@ -54,7 +55,7 @@ abstract class NodeMembershipSpec
awaitAssert(clusterView.members.size should ===(3)) awaitAssert(clusterView.members.size should ===(3))
assertMembers(clusterView.members, first, second, third) assertMembers(clusterView.members, first, second, third)
awaitAssert(clusterView.members.map(_.status) should ===(Set(MemberStatus.Up))) awaitAssert(clusterView.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up)))
enterBarrier("after-2") enterBarrier("after-2")
} }

View file

@ -19,6 +19,7 @@ import akka.actor.Actor
import akka.actor.RootActorPath import akka.actor.RootActorPath
import akka.cluster.MemberStatus._ import akka.cluster.MemberStatus._
import akka.actor.Deploy import akka.actor.Deploy
import akka.util.ccompat.imm._
object RestartFirstSeedNodeMultiJvmSpec extends MultiNodeConfig { object RestartFirstSeedNodeMultiJvmSpec extends MultiNodeConfig {
val seed1 = role("seed1") val seed1 = role("seed1")
@ -97,7 +98,7 @@ abstract class RestartFirstSeedNodeSpec
runOn(seed1) { runOn(seed1) {
Cluster(seed1System).joinSeedNodes(seedNodes) Cluster(seed1System).joinSeedNodes(seedNodes)
awaitAssert(Cluster(seed1System).readView.members.size should ===(3)) awaitAssert(Cluster(seed1System).readView.members.size should ===(3))
awaitAssert(Cluster(seed1System).readView.members.map(_.status) should ===(Set(Up))) awaitAssert(Cluster(seed1System).readView.members.unsorted.map(_.status) should ===(Set(Up)))
} }
runOn(seed2, seed3) { runOn(seed2, seed3) {
cluster.joinSeedNodes(seedNodes) cluster.joinSeedNodes(seedNodes)
@ -116,7 +117,7 @@ abstract class RestartFirstSeedNodeSpec
Cluster(restartedSeed1System).joinSeedNodes(seedNodes) Cluster(restartedSeed1System).joinSeedNodes(seedNodes)
within(20.seconds) { within(20.seconds) {
awaitAssert(Cluster(restartedSeed1System).readView.members.size should ===(3)) awaitAssert(Cluster(restartedSeed1System).readView.members.size should ===(3))
awaitAssert(Cluster(restartedSeed1System).readView.members.map(_.status) should ===(Set(Up))) awaitAssert(Cluster(restartedSeed1System).readView.members.unsorted.map(_.status) should ===(Set(Up)))
} }
} }
runOn(seed2, seed3) { runOn(seed2, seed3) {

View file

@ -18,6 +18,7 @@ import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec import akka.remote.testkit.MultiNodeSpec
import akka.testkit._ import akka.testkit._
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import akka.util.ccompat.imm._
object RestartNode2SpecMultiJvmSpec extends MultiNodeConfig { object RestartNode2SpecMultiJvmSpec extends MultiNodeConfig {
val seed1 = role("seed1") val seed1 = role("seed1")
@ -99,7 +100,7 @@ abstract class RestartNode2SpecSpec
runOn(seed1) { runOn(seed1) {
Cluster(seed1System).joinSeedNodes(seedNodes) Cluster(seed1System).joinSeedNodes(seedNodes)
awaitAssert(Cluster(seed1System).readView.members.size should be(2)) awaitAssert(Cluster(seed1System).readView.members.size should be(2))
awaitAssert(Cluster(seed1System).readView.members.map(_.status) should be(Set(Up))) awaitAssert(Cluster(seed1System).readView.members.unsorted.map(_.status) should be(Set(Up)))
} }
runOn(seed2) { runOn(seed2) {
cluster.joinSeedNodes(seedNodes) cluster.joinSeedNodes(seedNodes)
@ -118,7 +119,7 @@ abstract class RestartNode2SpecSpec
Cluster(restartedSeed1System).joinSeedNodes(seedNodes) Cluster(restartedSeed1System).joinSeedNodes(seedNodes)
within(30.seconds) { within(30.seconds) {
awaitAssert(Cluster(restartedSeed1System).readView.members.size should be(2)) awaitAssert(Cluster(restartedSeed1System).readView.members.size should be(2))
awaitAssert(Cluster(restartedSeed1System).readView.members.map(_.status) should be(Set(Up))) awaitAssert(Cluster(restartedSeed1System).readView.members.unsorted.map(_.status) should be(Set(Up)))
} }
} }
runOn(seed2) { runOn(seed2) {

View file

@ -18,6 +18,7 @@ import akka.remote.testkit.MultiNodeSpec
import akka.remote.transport.ThrottlerTransportAdapter.Direction import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.testkit._ import akka.testkit._
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import akka.util.ccompat.imm._
object RestartNode3MultiJvmSpec extends MultiNodeConfig { object RestartNode3MultiJvmSpec extends MultiNodeConfig {
val first = role("first") val first = role("first")
@ -133,7 +134,7 @@ abstract class RestartNode3Spec
runOn(second) { runOn(second) {
Cluster(restartedSecondSystem).joinSeedNodes(seedNodes) Cluster(restartedSecondSystem).joinSeedNodes(seedNodes)
awaitAssert(Cluster(restartedSecondSystem).readView.members.size should ===(3)) awaitAssert(Cluster(restartedSecondSystem).readView.members.size should ===(3))
awaitAssert(Cluster(restartedSecondSystem).readView.members.map(_.status) should ===(Set(Up))) awaitAssert(Cluster(restartedSecondSystem).readView.members.unsorted.map(_.status) should ===(Set(Up)))
} }
runOn(first, third) { runOn(first, third) {
awaitAssert { awaitAssert {

View file

@ -23,6 +23,7 @@ import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec import akka.remote.testkit.MultiNodeSpec
import akka.testkit._ import akka.testkit._
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import akka.util.ccompat.imm._
object RestartNodeMultiJvmSpec extends MultiNodeConfig { object RestartNodeMultiJvmSpec extends MultiNodeConfig {
val first = role("first") val first = role("first")
@ -121,7 +122,7 @@ abstract class RestartNodeSpec
runOn(second) { runOn(second) {
Cluster(secondSystem).joinSeedNodes(seedNodes) Cluster(secondSystem).joinSeedNodes(seedNodes)
awaitAssert(Cluster(secondSystem).readView.members.size should ===(3)) awaitAssert(Cluster(secondSystem).readView.members.size should ===(3))
awaitAssert(Cluster(secondSystem).readView.members.map(_.status) should ===(Set(Up))) awaitAssert(Cluster(secondSystem).readView.members.unsorted.map(_.status) should ===(Set(Up)))
} }
enterBarrier("started") enterBarrier("started")
@ -139,7 +140,7 @@ abstract class RestartNodeSpec
runOn(second) { runOn(second) {
Cluster(restartedSecondSystem).joinSeedNodes(seedNodes) Cluster(restartedSecondSystem).joinSeedNodes(seedNodes)
awaitAssert(Cluster(restartedSecondSystem).readView.members.size should ===(3)) awaitAssert(Cluster(restartedSecondSystem).readView.members.size should ===(3))
awaitAssert(Cluster(restartedSecondSystem).readView.members.map(_.status) should ===(Set(Up))) awaitAssert(Cluster(restartedSecondSystem).readView.members.unsorted.map(_.status) should ===(Set(Up)))
} }
runOn(first, third) { runOn(first, third) {
awaitAssert { awaitAssert {

View file

@ -345,7 +345,7 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig {
class PhiObserver extends Actor with ActorLogging { class PhiObserver extends Actor with ActorLogging {
val cluster = Cluster(context.system) val cluster = Cluster(context.system)
var reportTo: Option[ActorRef] = None var reportTo: Option[ActorRef] = None
val emptyPhiByNode = Map.empty[Address, PhiValue].withDefault(address PhiValue(address, 0, 0, 0.0)) val emptyPhiByNode: Map[Address, PhiValue] = Map.empty[Address, PhiValue].withDefault(address PhiValue(address, 0, 0, 0.0))
var phiByNode = emptyPhiByNode var phiByNode = emptyPhiByNode
var nodes = Set.empty[Address] var nodes = Set.empty[Address]

View file

@ -19,6 +19,7 @@ import akka.testkit._
import akka.actor.Props import akka.actor.Props
import akka.cluster.MultiNodeClusterSpec.EndActor import akka.cluster.MultiNodeClusterSpec.EndActor
import akka.remote.RARP import akka.remote.RARP
import akka.util.ccompat.imm._
object UnreachableNodeJoinsAgainMultiNodeConfig extends MultiNodeConfig { object UnreachableNodeJoinsAgainMultiNodeConfig extends MultiNodeConfig {
val first = role("first") val first = role("first")
@ -185,7 +186,7 @@ abstract class UnreachableNodeJoinsAgainSpec
within(30 seconds) { within(30 seconds) {
awaitAssert(Cluster(freshSystem).readView.members.map(_.address) should contain(victimAddress)) awaitAssert(Cluster(freshSystem).readView.members.map(_.address) should contain(victimAddress))
awaitAssert(Cluster(freshSystem).readView.members.size should ===(expectedNumberOfMembers)) awaitAssert(Cluster(freshSystem).readView.members.size should ===(expectedNumberOfMembers))
awaitAssert(Cluster(freshSystem).readView.members.map(_.status) should ===(Set(MemberStatus.Up))) awaitAssert(Cluster(freshSystem).readView.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up)))
} }
// signal to master node that victim is done // signal to master node that victim is done

View file

@ -37,7 +37,7 @@ class ClusterDomainEventSpec extends WordSpec with Matchers {
val selfDummyAddress = UniqueAddress(Address("akka.tcp", "sys", "selfDummy", 2552), 17L) val selfDummyAddress = UniqueAddress(Address("akka.tcp", "sys", "selfDummy", 2552), 17L)
private[cluster] def converge(gossip: Gossip): (Gossip, Set[UniqueAddress]) = private[cluster] def converge(gossip: Gossip): (Gossip, Set[UniqueAddress]) =
((gossip, Set.empty[UniqueAddress]) /: gossip.members) { case ((gs, as), m) (gs.seen(m.uniqueAddress), as + m.uniqueAddress) } gossip.members.foldLeft((gossip, Set.empty[UniqueAddress])) { case ((gs, as), m) (gs.seen(m.uniqueAddress), as + m.uniqueAddress) }
private def state(g: Gossip): MembershipState = private def state(g: Gossip): MembershipState =
state(g, selfDummyAddress) state(g, selfDummyAddress)

View file

@ -18,7 +18,7 @@ class ReachabilityPerfSpec extends WordSpec with Matchers {
val node = Address("akka.tcp", "sys", "a", 2552) val node = Address("akka.tcp", "sys", "a", 2552)
private def createReachabilityOfSize(base: Reachability, size: Int): Reachability = private def createReachabilityOfSize(base: Reachability, size: Int): Reachability =
(base /: (1 to size)) { (1 to size).foldLeft(base) {
case (r, i) case (r, i)
val observer = UniqueAddress(address.copy(host = Some("node-" + i)), i.toLong) val observer = UniqueAddress(address.copy(host = Some("node-" + i)), i.toLong)
val j = if (i == size) 1 else i + 1 val j = if (i == size) 1 else i + 1
@ -29,9 +29,9 @@ class ReachabilityPerfSpec extends WordSpec with Matchers {
private def addUnreachable(base: Reachability, count: Int): Reachability = { private def addUnreachable(base: Reachability, count: Int): Reachability = {
val observers = base.allObservers.take(count) val observers = base.allObservers.take(count)
val subjects = Stream.continually(base.allObservers).flatten.iterator val subjects = Stream.continually(base.allObservers).flatten.iterator
(base /: observers) { observers.foldLeft(base) {
case (r, o) case (r, o)
(r /: (1 to 5)) { case (r, _) r.unreachable(o, subjects.next()) } (1 to 5).foldLeft(r) { case (r, _) r.unreachable(o, subjects.next()) }
} }
} }

View file

@ -12,14 +12,14 @@ object VectorClockPerfSpec {
import VectorClock._ import VectorClock._
def createVectorClockOfSize(size: Int): (VectorClock, SortedSet[Node]) = def createVectorClockOfSize(size: Int): (VectorClock, SortedSet[Node]) =
((VectorClock(), SortedSet.empty[Node]) /: (1 to size)) { (1 to size).foldLeft((VectorClock(), SortedSet.empty[Node])) {
case ((vc, nodes), i) case ((vc, nodes), i)
val node = Node(i.toString) val node = Node(i.toString)
(vc :+ node, nodes + node) (vc :+ node, nodes + node)
} }
def copyVectorClock(vc: VectorClock): VectorClock = { def copyVectorClock(vc: VectorClock): VectorClock = {
val versions = (TreeMap.empty[Node, Long] /: vc.versions) { val versions = vc.versions.foldLeft(TreeMap.empty[Node, Long]) {
case (versions, (n, t)) versions.updated(Node.fromHash(n), t) case (versions, (n, t)) versions.updated(Node.fromHash(n), t)
} }
vc.copy(versions = versions) vc.copy(versions = versions)

View file

@ -98,9 +98,7 @@ trait JavaLoggingAdapter extends LoggingAdapter {
if (loggingExecutionContext.isDefined) { if (loggingExecutionContext.isDefined) {
implicit val context = loggingExecutionContext.get implicit val context = loggingExecutionContext.get
Future(logger.log(record)).onFailure { Future(logger.log(record)).failed.foreach { _.printStackTrace() }
case thrown: Throwable thrown.printStackTrace()
}
} else } else
logger.log(record) logger.log(record)
} }

View file

@ -11,6 +11,7 @@ import akka.annotation.InternalApi
import akka.cluster.ddata.Key.KeyId import akka.cluster.ddata.Key.KeyId
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation import akka.cluster.ddata.Replicator.Internal.DeltaPropagation
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation.NoDeltaPlaceholder import akka.cluster.ddata.Replicator.Internal.DeltaPropagation.NoDeltaPlaceholder
import akka.util.ccompat._
/** /**
* INTERNAL API: Used by the Replicator actor. * INTERNAL API: Used by the Replicator actor.
@ -144,7 +145,7 @@ import akka.cluster.ddata.Replicator.Internal.DeltaPropagation.NoDeltaPlaceholde
} }
private def deltaEntriesAfter(entries: TreeMap[Long, ReplicatedData], version: Long): TreeMap[Long, ReplicatedData] = private def deltaEntriesAfter(entries: TreeMap[Long, ReplicatedData], version: Long): TreeMap[Long, ReplicatedData] =
entries.from(version) match { entries.rangeFrom(version) match {
case ntrs if ntrs.isEmpty ntrs case ntrs if ntrs.isEmpty ntrs
case ntrs if ntrs.firstKey == version ntrs.tail // exclude first, i.e. version j that was already sent case ntrs if ntrs.firstKey == version ntrs.tail // exclude first, i.e. version j that was already sent
case ntrs ntrs case ntrs ntrs

View file

@ -53,6 +53,7 @@ import scala.collection.immutable.TreeSet
import akka.cluster.MemberStatus import akka.cluster.MemberStatus
import scala.annotation.varargs import scala.annotation.varargs
import akka.util.JavaDurationConverters._ import akka.util.JavaDurationConverters._
import akka.util.ccompat._
object ReplicatorSettings { object ReplicatorSettings {
@ -1109,13 +1110,13 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
override def createDeltaPropagation(deltas: Map[KeyId, (ReplicatedData, Long, Long)]): DeltaPropagation = { override def createDeltaPropagation(deltas: Map[KeyId, (ReplicatedData, Long, Long)]): DeltaPropagation = {
// Important to include the pruning state in the deltas. For example if the delta is based // Important to include the pruning state in the deltas. For example if the delta is based
// on an entry that has been pruned but that has not yet been performed on the target node. // on an entry that has been pruned but that has not yet been performed on the target node.
DeltaPropagation(selfUniqueAddress, reply = false, deltas.collect { DeltaPropagation(selfUniqueAddress, reply = false, deltas.iterator.collect {
case (key, (d, fromSeqNr, toSeqNr)) if d != NoDeltaPlaceholder case (key, (d, fromSeqNr, toSeqNr)) if d != NoDeltaPlaceholder
getData(key) match { getData(key) match {
case Some(envelope) key Delta(envelope.copy(data = d), fromSeqNr, toSeqNr) case Some(envelope) key Delta(envelope.copy(data = d), fromSeqNr, toSeqNr)
case None key Delta(DataEnvelope(d), fromSeqNr, toSeqNr) case None key Delta(DataEnvelope(d), fromSeqNr, toSeqNr)
} }
}(collection.breakOut)) }.toMap)
} }
} }
val deltaPropagationTask: Option[Cancellable] = val deltaPropagationTask: Option[Cancellable] =
@ -1461,9 +1462,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
} }
def receiveGetKeyIds(): Unit = { def receiveGetKeyIds(): Unit = {
val keys: Set[KeyId] = dataEntries.collect { val keys: Set[KeyId] = dataEntries.iterator.collect {
case (key, (DataEnvelope(data, _, _), _)) if data != DeletedData key case (key, (DataEnvelope(data, _, _), _)) if data != DeletedData key
}(collection.breakOut) }.to(immutable.Set)
replyTo ! GetKeyIdsResult(keys) replyTo ! GetKeyIdsResult(keys)
} }
@ -1700,14 +1701,14 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
if (keys.nonEmpty) { if (keys.nonEmpty) {
if (log.isDebugEnabled) if (log.isDebugEnabled)
log.debug("Sending gossip to [{}], containing [{}]", replyTo.path.address, keys.mkString(", ")) log.debug("Sending gossip to [{}], containing [{}]", replyTo.path.address, keys.mkString(", "))
val g = Gossip(keys.map(k k getData(k).get)(collection.breakOut), sendBack = otherDifferentKeys.nonEmpty) val g = Gossip(keys.iterator.map(k k getData(k).get).toMap, sendBack = otherDifferentKeys.nonEmpty)
replyTo ! g replyTo ! g
} }
val myMissingKeys = otherKeys diff myKeys val myMissingKeys = otherKeys diff myKeys
if (myMissingKeys.nonEmpty) { if (myMissingKeys.nonEmpty) {
if (log.isDebugEnabled) if (log.isDebugEnabled)
log.debug("Sending gossip status to [{}], requesting missing [{}]", replyTo.path.address, myMissingKeys.mkString(", ")) log.debug("Sending gossip status to [{}], requesting missing [{}]", replyTo.path.address, myMissingKeys.mkString(", "))
val status = Status(myMissingKeys.map(k k NotFoundDigest)(collection.breakOut), chunk, totChunks) val status = Status(myMissingKeys.iterator.map(k k NotFoundDigest).toMap, chunk, totChunks)
replyTo ! status replyTo ! status
} }
} }
@ -1846,9 +1847,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
def initRemovedNodePruning(): Unit = { def initRemovedNodePruning(): Unit = {
// initiate pruning for removed nodes // initiate pruning for removed nodes
val removedSet: Set[UniqueAddress] = removedNodes.collect { val removedSet: Set[UniqueAddress] = removedNodes.iterator.collect {
case (r, t) if ((allReachableClockTime - t) > maxPruningDisseminationNanos) r case (r, t) if ((allReachableClockTime - t) > maxPruningDisseminationNanos) r
}(collection.breakOut) }.to(immutable.Set)
if (removedSet.nonEmpty) { if (removedSet.nonEmpty) {
for ((key, (envelope, _)) dataEntries; removed removedSet) { for ((key, (envelope, _)) dataEntries; removed removedSet) {

View file

@ -12,7 +12,7 @@ import java.util.TreeSet
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
import scala.collection.breakOut import scala.collection.immutable
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
import akka.cluster.ddata._ import akka.cluster.ddata._
@ -28,6 +28,7 @@ import java.io.NotSerializableException
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.cluster.ddata.protobuf.msg.ReplicatorMessages.OtherMessage import akka.cluster.ddata.protobuf.msg.ReplicatorMessages.OtherMessage
import akka.serialization.Serialization import akka.serialization.Serialization
import akka.util.ccompat._
private object ReplicatedDataSerializer { private object ReplicatedDataSerializer {
/* /*
@ -472,7 +473,7 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
private def orsetDeltaGroupFromBinary(bytes: Array[Byte]): ORSet.DeltaGroup[Any] = { private def orsetDeltaGroupFromBinary(bytes: Array[Byte]): ORSet.DeltaGroup[Any] = {
val deltaGroup = rd.ORSetDeltaGroup.parseFrom(bytes) val deltaGroup = rd.ORSetDeltaGroup.parseFrom(bytes)
val ops: Vector[ORSet.DeltaOp] = val ops: Vector[ORSet.DeltaOp] =
deltaGroup.getEntriesList.asScala.map { entry deltaGroup.getEntriesList.asScala.iterator.map { entry
if (entry.getOperation == rd.ORSetDeltaOp.Add) if (entry.getOperation == rd.ORSetDeltaOp.Add)
ORSet.AddDeltaOp(orsetFromProto(entry.getUnderlying)) ORSet.AddDeltaOp(orsetFromProto(entry.getUnderlying))
else if (entry.getOperation == rd.ORSetDeltaOp.Remove) else if (entry.getOperation == rd.ORSetDeltaOp.Remove)
@ -481,7 +482,7 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
ORSet.FullStateDeltaOp(orsetFromProto(entry.getUnderlying)) ORSet.FullStateDeltaOp(orsetFromProto(entry.getUnderlying))
else else
throw new NotSerializableException(s"Unknow ORSet delta operation ${entry.getOperation}") throw new NotSerializableException(s"Unknow ORSet delta operation ${entry.getOperation}")
}(collection.breakOut) }.to(immutable.Vector)
ORSet.DeltaGroup(ops) ORSet.DeltaGroup(ops)
} }
@ -538,8 +539,8 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
gcounterFromProto(rd.GCounter.parseFrom(bytes)) gcounterFromProto(rd.GCounter.parseFrom(bytes))
def gcounterFromProto(gcounter: rd.GCounter): GCounter = { def gcounterFromProto(gcounter: rd.GCounter): GCounter = {
new GCounter(state = gcounter.getEntriesList.asScala.map(entry new GCounter(state = gcounter.getEntriesList.asScala.iterator.map(entry
uniqueAddressFromProto(entry.getNode) BigInt(entry.getValue.toByteArray))(breakOut)) uniqueAddressFromProto(entry.getNode) BigInt(entry.getValue.toByteArray)).toMap)
} }
def pncounterToProto(pncounter: PNCounter): rd.PNCounter = def pncounterToProto(pncounter: PNCounter): rd.PNCounter =
@ -670,7 +671,7 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
private def ormapDeltaGroupOpsFromBinary(bytes: Array[Byte]): scala.collection.immutable.IndexedSeq[ORMap.DeltaOp] = { private def ormapDeltaGroupOpsFromBinary(bytes: Array[Byte]): scala.collection.immutable.IndexedSeq[ORMap.DeltaOp] = {
val deltaGroup = rd.ORMapDeltaGroup.parseFrom(bytes) val deltaGroup = rd.ORMapDeltaGroup.parseFrom(bytes)
val ops: Vector[ORMap.DeltaOp] = val ops: Vector[ORMap.DeltaOp] =
deltaGroup.getEntriesList.asScala.map { entry deltaGroup.getEntriesList.asScala.iterator.map { entry
if (entry.getOperation == rd.ORMapDeltaOp.ORMapPut) { if (entry.getOperation == rd.ORMapDeltaOp.ORMapPut) {
val map = singleMapEntryFromProto(entry.getEntryDataList, (v: dm.OtherMessage) otherMessageFromProto(v).asInstanceOf[ReplicatedData]) val map = singleMapEntryFromProto(entry.getEntryDataList, (v: dm.OtherMessage) otherMessageFromProto(v).asInstanceOf[ReplicatedData])
ORMap.PutDeltaOp(ORSet.AddDeltaOp(orsetFromProto(entry.getUnderlying)), map.head, zeroTagFromCode(entry.getZeroTag)) ORMap.PutDeltaOp(ORSet.AddDeltaOp(orsetFromProto(entry.getUnderlying)), map.head, zeroTagFromCode(entry.getZeroTag))
@ -684,31 +685,31 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
ORMap.UpdateDeltaOp(ORSet.AddDeltaOp(orsetFromProto(entry.getUnderlying)), map, zeroTagFromCode(entry.getZeroTag)) ORMap.UpdateDeltaOp(ORSet.AddDeltaOp(orsetFromProto(entry.getUnderlying)), map, zeroTagFromCode(entry.getZeroTag))
} else } else
throw new NotSerializableException(s"Unknown ORMap delta operation ${entry.getOperation}") throw new NotSerializableException(s"Unknown ORMap delta operation ${entry.getOperation}")
}(collection.breakOut) }.to(immutable.Vector)
ops ops
} }
private def ormapPutToProto(deltaOp: ORMap.PutDeltaOp[_, _]): rd.ORMapDeltaGroup = { private def ormapPutToProto(deltaOp: ORMap.PutDeltaOp[_, _]): rd.ORMapDeltaGroup = {
ormapDeltaGroupOpsToProto(scala.collection.immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp])) ormapDeltaGroupOpsToProto(immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
} }
private def ormapRemoveToProto(deltaOp: ORMap.RemoveDeltaOp[_, _]): rd.ORMapDeltaGroup = { private def ormapRemoveToProto(deltaOp: ORMap.RemoveDeltaOp[_, _]): rd.ORMapDeltaGroup = {
ormapDeltaGroupOpsToProto(scala.collection.immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp])) ormapDeltaGroupOpsToProto(immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
} }
private def ormapRemoveKeyToProto(deltaOp: ORMap.RemoveKeyDeltaOp[_, _]): rd.ORMapDeltaGroup = { private def ormapRemoveKeyToProto(deltaOp: ORMap.RemoveKeyDeltaOp[_, _]): rd.ORMapDeltaGroup = {
ormapDeltaGroupOpsToProto(scala.collection.immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp])) ormapDeltaGroupOpsToProto(immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
} }
private def ormapUpdateToProto(deltaOp: ORMap.UpdateDeltaOp[_, _]): rd.ORMapDeltaGroup = { private def ormapUpdateToProto(deltaOp: ORMap.UpdateDeltaOp[_, _]): rd.ORMapDeltaGroup = {
ormapDeltaGroupOpsToProto(scala.collection.immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp])) ormapDeltaGroupOpsToProto(immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
} }
private def ormapDeltaGroupToProto(deltaGroup: ORMap.DeltaGroup[_, _]): rd.ORMapDeltaGroup = { private def ormapDeltaGroupToProto(deltaGroup: ORMap.DeltaGroup[_, _]): rd.ORMapDeltaGroup = {
ormapDeltaGroupOpsToProto(deltaGroup.ops) ormapDeltaGroupOpsToProto(deltaGroup.ops)
} }
private def ormapDeltaGroupOpsToProto(deltaGroupOps: scala.collection.immutable.IndexedSeq[ORMap.DeltaOp]): rd.ORMapDeltaGroup = { private def ormapDeltaGroupOpsToProto(deltaGroupOps: immutable.IndexedSeq[ORMap.DeltaOp]): rd.ORMapDeltaGroup = {
def createEntry(opType: rd.ORMapDeltaOp, u: ORSet[_], m: Map[_, _], zt: Int) = { def createEntry(opType: rd.ORMapDeltaOp, u: ORSet[_], m: Map[_, _], zt: Int) = {
if (m.size > 1 && opType != rd.ORMapDeltaOp.ORMapUpdate) if (m.size > 1 && opType != rd.ORMapDeltaOp.ORMapUpdate)
throw new IllegalArgumentException("Invalid size of ORMap delta map") throw new IllegalArgumentException("Invalid size of ORMap delta map")

View file

@ -7,7 +7,7 @@ package akka.cluster.ddata.protobuf
import scala.concurrent.duration._ import scala.concurrent.duration._
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
import scala.collection.breakOut import scala.collection.immutable
import scala.concurrent.duration.Duration import scala.concurrent.duration.Duration
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
import akka.cluster.Member import akka.cluster.Member
@ -32,6 +32,7 @@ import akka.actor.Address
import akka.cluster.ddata.VersionVector import akka.cluster.ddata.VersionVector
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.cluster.ddata.PruningState.PruningPerformed import akka.cluster.ddata.PruningState.PruningPerformed
import akka.util.ccompat._
/** /**
* INTERNAL API * INTERNAL API
@ -268,8 +269,8 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
private def statusFromBinary(bytes: Array[Byte]): Status = { private def statusFromBinary(bytes: Array[Byte]): Status = {
val status = dm.Status.parseFrom(bytes) val status = dm.Status.parseFrom(bytes)
Status( Status(
status.getEntriesList.asScala.map(e status.getEntriesList.asScala.iterator.map(e
e.getKey AkkaByteString(e.getDigest.toByteArray()))(breakOut), e.getKey AkkaByteString(e.getDigest.toByteArray())).toMap,
status.getChunk, status.getTotChunks) status.getChunk, status.getTotChunks)
} }
@ -287,8 +288,8 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
private def gossipFromBinary(bytes: Array[Byte]): Gossip = { private def gossipFromBinary(bytes: Array[Byte]): Gossip = {
val gossip = dm.Gossip.parseFrom(decompress(bytes)) val gossip = dm.Gossip.parseFrom(decompress(bytes))
Gossip( Gossip(
gossip.getEntriesList.asScala.map(e gossip.getEntriesList.asScala.iterator.map(e
e.getKey dataEnvelopeFromProto(e.getEnvelope))(breakOut), e.getKey dataEnvelopeFromProto(e.getEnvelope)).toMap,
sendBack = gossip.getSendBack) sendBack = gossip.getSendBack)
} }
@ -316,11 +317,11 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
DeltaPropagation( DeltaPropagation(
uniqueAddressFromProto(deltaPropagation.getFromNode), uniqueAddressFromProto(deltaPropagation.getFromNode),
reply, reply,
deltaPropagation.getEntriesList.asScala.map { e deltaPropagation.getEntriesList.asScala.iterator.map { e
val fromSeqNr = e.getFromSeqNr val fromSeqNr = e.getFromSeqNr
val toSeqNr = if (e.hasToSeqNr) e.getToSeqNr else fromSeqNr val toSeqNr = if (e.hasToSeqNr) e.getToSeqNr else fromSeqNr
e.getKey Delta(dataEnvelopeFromProto(e.getEnvelope), fromSeqNr, toSeqNr) e.getKey Delta(dataEnvelopeFromProto(e.getEnvelope), fromSeqNr, toSeqNr)
}(breakOut)) }.toMap)
} }
private def getToProto(get: Get[_]): dm.Get = { private def getToProto(get: Get[_]): dm.Get = {
@ -482,7 +483,7 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
if (pruningEntries.isEmpty) if (pruningEntries.isEmpty)
Map.empty Map.empty
else else
pruningEntries.asScala.map { pruningEntry pruningEntries.asScala.iterator.map { pruningEntry
val state = val state =
if (pruningEntry.getPerformed) { if (pruningEntry.getPerformed) {
// for wire compatibility with Akka 2.4.x // for wire compatibility with Akka 2.4.x
@ -491,10 +492,10 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
} else } else
PruningState.PruningInitialized( PruningState.PruningInitialized(
uniqueAddressFromProto(pruningEntry.getOwnerAddress), uniqueAddressFromProto(pruningEntry.getOwnerAddress),
pruningEntry.getSeenList.asScala.map(addressFromProto)(breakOut)) pruningEntry.getSeenList.asScala.iterator.map(addressFromProto).to(immutable.Set))
val removed = uniqueAddressFromProto(pruningEntry.getRemovedAddress) val removed = uniqueAddressFromProto(pruningEntry.getRemovedAddress)
removed state removed state
}(breakOut) }.toMap
} }
private def writeToProto(write: Write): dm.Write = private def writeToProto(write: Write): dm.Write =

View file

@ -11,7 +11,6 @@ import java.util.zip.GZIPOutputStream
import scala.annotation.tailrec import scala.annotation.tailrec
import scala.collection.immutable.TreeMap import scala.collection.immutable.TreeMap
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
import scala.collection.breakOut
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.actor.Address import akka.actor.Address
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
@ -21,6 +20,7 @@ import akka.serialization._
import akka.protobuf.ByteString import akka.protobuf.ByteString
import akka.protobuf.MessageLite import akka.protobuf.MessageLite
import akka.cluster.ddata.VersionVector import akka.cluster.ddata.VersionVector
import akka.util.ccompat._
/** /**
* Some useful serialization helper methods. * Some useful serialization helper methods.
@ -124,8 +124,8 @@ trait SerializationSupport {
else if (entries.size == 1) else if (entries.size == 1)
VersionVector(uniqueAddressFromProto(entries.get(0).getNode), entries.get(0).getVersion) VersionVector(uniqueAddressFromProto(entries.get(0).getNode), entries.get(0).getVersion)
else { else {
val versions: TreeMap[UniqueAddress, Long] = versionVector.getEntriesList.asScala.map(entry val versions: TreeMap[UniqueAddress, Long] = scala.collection.immutable.TreeMap.from(versionVector.getEntriesList.asScala.iterator.map(entry
uniqueAddressFromProto(entry.getNode) entry.getVersion)(breakOut) uniqueAddressFromProto(entry.getNode) entry.getVersion))
VersionVector(versions) VersionVector(versions)
} }
} }

View file

@ -136,7 +136,7 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig)
expectTerminated(r) expectTerminated(r)
var r2: ActorRef = null var r2: ActorRef = null
awaitAssert(r2 = newReplicator()) // try until name is free awaitAssert { r2 = newReplicator() } // try until name is free
// note that it will stash the commands until loading completed // note that it will stash the commands until loading completed
r2 ! Get(KeyA, ReadLocal) r2 ! Get(KeyA, ReadLocal)
@ -184,7 +184,7 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig)
expectTerminated(r) expectTerminated(r)
var r2: ActorRef = null var r2: ActorRef = null
awaitAssert(r2 = newReplicator()) // try until name is free awaitAssert { r2 = newReplicator() } // try until name is free
awaitAssert { awaitAssert {
r2 ! GetKeyIds r2 ! GetKeyIds
expectMsgType[GetKeyIdsResult].keyIds should !==(Set.empty[String]) expectMsgType[GetKeyIdsResult].keyIds should !==(Set.empty[String])
@ -221,7 +221,7 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig)
expectTerminated(r) expectTerminated(r)
var r2: ActorRef = null var r2: ActorRef = null
awaitAssert(r2 = newReplicator()) // try until name is free awaitAssert { r2 = newReplicator() } // try until name is free
awaitAssert { awaitAssert {
r2 ! GetKeyIds r2 ! GetKeyIds
expectMsgType[GetKeyIdsResult].keyIds should !==(Set.empty[String]) expectMsgType[GetKeyIdsResult].keyIds should !==(Set.empty[String])

View file

@ -16,6 +16,7 @@ import akka.actor.ActorSystem
import akka.actor.ActorRef import akka.actor.ActorRef
import scala.concurrent.Await import scala.concurrent.Await
import akka.cluster.MemberStatus import akka.cluster.MemberStatus
import akka.util.ccompat.imm._
object DurablePruningSpec extends MultiNodeConfig { object DurablePruningSpec extends MultiNodeConfig {
val first = role("first") val first = role("first")
@ -76,9 +77,9 @@ class DurablePruningSpec extends MultiNodeSpec(DurablePruningSpec) with STMultiN
Cluster(sys2).join(node(first).address) Cluster(sys2).join(node(first).address)
awaitAssert({ awaitAssert({
Cluster(system).state.members.size should ===(4) Cluster(system).state.members.size should ===(4)
Cluster(system).state.members.map(_.status) should ===(Set(MemberStatus.Up)) Cluster(system).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
Cluster(sys2).state.members.size should ===(4) Cluster(sys2).state.members.size should ===(4)
Cluster(sys2).state.members.map(_.status) should ===(Set(MemberStatus.Up)) Cluster(sys2).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
}, 10.seconds) }, 10.seconds)
enterBarrier("joined") enterBarrier("joined")

View file

@ -21,8 +21,8 @@ object DeltaPropagationSelectorSpec {
override val allNodes: Vector[Address]) extends DeltaPropagationSelector { override val allNodes: Vector[Address]) extends DeltaPropagationSelector {
override val gossipIntervalDivisor = 5 override val gossipIntervalDivisor = 5
override def createDeltaPropagation(deltas: Map[KeyId, (ReplicatedData, Long, Long)]): DeltaPropagation = override def createDeltaPropagation(deltas: Map[KeyId, (ReplicatedData, Long, Long)]): DeltaPropagation =
DeltaPropagation(selfUniqueAddress, false, deltas.mapValues { DeltaPropagation(selfUniqueAddress, false, deltas.map {
case (d, fromSeqNr, toSeqNr) Delta(DataEnvelope(d), fromSeqNr, toSeqNr) case (key, (d, fromSeqNr, toSeqNr)) (key, Delta(DataEnvelope(d), fromSeqNr, toSeqNr))
}) })
override def maxDeltaSize: Int = 10 override def maxDeltaSize: Int = 10
} }

View file

@ -24,8 +24,8 @@ class LWWMapSpec extends WordSpec with Matchers {
} }
"be able to have its entries correctly merged with another LWWMap with other entries" in { "be able to have its entries correctly merged with another LWWMap with other entries" in {
val m1 = LWWMap.empty.put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int]) val m1 = LWWMap.empty[String, Int].put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int])
val m2 = LWWMap.empty.put(node2, "c", 3, defaultClock[Int]) val m2 = LWWMap.empty[String, Int].put(node2, "c", 3, defaultClock[Int])
// merge both ways // merge both ways
val expected = Map("a" 1, "b" 2, "c" 3) val expected = Map("a" 1, "b" 2, "c" 3)
@ -34,8 +34,8 @@ class LWWMapSpec extends WordSpec with Matchers {
} }
"be able to remove entry" in { "be able to remove entry" in {
val m1 = LWWMap.empty.put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int]) val m1 = LWWMap.empty[String, Int].put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int])
val m2 = LWWMap.empty.put(node2, "c", 3, defaultClock[Int]) val m2 = LWWMap.empty[String, Int].put(node2, "c", 3, defaultClock[Int])
val merged1 = m1 merge m2 val merged1 = m1 merge m2
@ -48,8 +48,8 @@ class LWWMapSpec extends WordSpec with Matchers {
} }
"be able to work with deltas" in { "be able to work with deltas" in {
val m1 = LWWMap.empty.put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int]) val m1 = LWWMap.empty[String, Int].put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int])
val m2 = LWWMap.empty.put(node2, "c", 3, defaultClock[Int]) val m2 = LWWMap.empty[String, Int].put(node2, "c", 3, defaultClock[Int])
val expected = Map("a" 1, "b" 2, "c" 3) val expected = Map("a" 1, "b" 2, "c" 3)
(m1 merge m2).entries should be(expected) (m1 merge m2).entries should be(expected)
@ -69,7 +69,7 @@ class LWWMapSpec extends WordSpec with Matchers {
} }
"have unapply extractor" in { "have unapply extractor" in {
val m1 = LWWMap.empty.put(node1, "a", 1L, defaultClock[Long]) val m1 = LWWMap.empty[String, Long].put(node1, "a", 1L, defaultClock[Long])
val LWWMap(entries1) = m1 val LWWMap(entries1) = m1
val entries2: Map[String, Long] = entries1 val entries2: Map[String, Long] = entries1
Changed(LWWMapKey[String, Long]("key"))(m1) match { Changed(LWWMapKey[String, Long]("key"))(m1) match {

View file

@ -368,21 +368,7 @@ you can also parallelize it by chunking your futures into sub-sequences and redu
## Callbacks ## Callbacks
Sometimes you just want to listen to a `Future` being completed, and react to that not by creating a new `Future`, but by side-effecting. Sometimes you just want to listen to a `Future` being completed, and react to that not by creating a new `Future`, but by side-effecting.
For this `Future` supports `onComplete`, `onSuccess` and `onFailure`, of which the last two are specializations of the first. For this, `Future` supports `onComplete`:
Scala
: @@snip [FutureDocSpec.scala](/akka-docs/src/test/scala/docs/future/FutureDocSpec.scala) { #onSuccess }
Java
: @@snip [FutureDocTest.java](/akka-docs/src/test/java/jdocs/future/FutureDocTest.java) { #onSuccess }
Scala
: @@snip [FutureDocSpec.scala](/akka-docs/src/test/scala/docs/future/FutureDocSpec.scala) { #onFailure }
Java
: @@snip [FutureDocTest.java](/akka-docs/src/test/java/jdocs/future/FutureDocTest.java) { #onFailure }
Scala Scala
: @@snip [FutureDocSpec.scala](/akka-docs/src/test/scala/docs/future/FutureDocSpec.scala) { #onComplete } : @@snip [FutureDocSpec.scala](/akka-docs/src/test/scala/docs/future/FutureDocSpec.scala) { #onComplete }

View file

@ -18,14 +18,15 @@ import akka.util.Timeout;
//#imports2 //#imports2
import java.time.Duration; import java.time.Duration;
import akka.japi.Function;
import java.util.concurrent.*; import java.util.concurrent.*;
import scala.util.Try;
import akka.japi.Function;
import static akka.dispatch.Futures.future; import static akka.dispatch.Futures.future;
import static java.util.concurrent.TimeUnit.SECONDS; import static java.util.concurrent.TimeUnit.SECONDS;
//#imports2 //#imports2
//#imports3 //#imports3
@ -283,7 +284,7 @@ public class FutureDocTest extends AbstractJavaTest {
} }
}, system.dispatcher()); }, system.dispatcher());
f.onSuccess(new PrintResult<String>(), system.dispatcher()); f.onComplete(new PrintResult<Try<String>>(), system.dispatcher());
//#future-eval //#future-eval
Timeout timeout = Timeout.create(Duration.ofSeconds(5)); Timeout timeout = Timeout.create(Duration.ofSeconds(5));
String result = (String) Await.result(f, timeout.duration()); String result = (String) Await.result(f, timeout.duration());
@ -307,7 +308,7 @@ public class FutureDocTest extends AbstractJavaTest {
} }
}, ec); }, ec);
f2.onSuccess(new PrintResult<Integer>(), system.dispatcher()); f2.onComplete(new PrintResult<Try<Integer>>(), system.dispatcher());
//#map //#map
Timeout timeout = Timeout.create(Duration.ofSeconds(5)); Timeout timeout = Timeout.create(Duration.ofSeconds(5));
int result = Await.result(f2, timeout.duration()); int result = Await.result(f2, timeout.duration());
@ -335,7 +336,7 @@ public class FutureDocTest extends AbstractJavaTest {
} }
}, ec); }, ec);
f2.onSuccess(new PrintResult<Integer>(), system.dispatcher()); f2.onComplete(new PrintResult<Try<Integer>>(), system.dispatcher());
//#flat-map //#flat-map
Timeout timeout = Timeout.create(Duration.ofSeconds(5)); Timeout timeout = Timeout.create(Duration.ofSeconds(5));
int result = Await.result(f2, timeout.duration()); int result = Await.result(f2, timeout.duration());
@ -367,7 +368,7 @@ public class FutureDocTest extends AbstractJavaTest {
} }
}, ec); }, ec);
futureSum.onSuccess(new PrintResult<Long>(), system.dispatcher()); futureSum.onComplete(new PrintResult<Try<Long>>(), system.dispatcher());
//#sequence //#sequence
Timeout timeout = Timeout.create(Duration.ofSeconds(5)); Timeout timeout = Timeout.create(Duration.ofSeconds(5));
long result = Await.result(futureSum, timeout.duration()); long result = Await.result(futureSum, timeout.duration());
@ -393,7 +394,7 @@ public class FutureDocTest extends AbstractJavaTest {
}, ec); }, ec);
//Returns the sequence of strings as upper case //Returns the sequence of strings as upper case
futureResult.onSuccess(new PrintResult<Iterable<String>>(), system.dispatcher()); futureResult.onComplete(new PrintResult<Try<Iterable<String>>>(), system.dispatcher());
//#traverse //#traverse
Timeout timeout = Timeout.create(Duration.ofSeconds(5)); Timeout timeout = Timeout.create(Duration.ofSeconds(5));
Iterable<String> result = Await.result(futureResult, timeout.duration()); Iterable<String> result = Await.result(futureResult, timeout.duration());
@ -420,7 +421,7 @@ public class FutureDocTest extends AbstractJavaTest {
} }
}, ec); }, ec);
resultFuture.onSuccess(new PrintResult<String>(), system.dispatcher()); resultFuture.onComplete(new PrintResult<Try<String>>(), system.dispatcher());
//#fold //#fold
Timeout timeout = Timeout.create(Duration.ofSeconds(5)); Timeout timeout = Timeout.create(Duration.ofSeconds(5));
String result = Await.result(resultFuture, timeout.duration()); String result = Await.result(resultFuture, timeout.duration());
@ -445,7 +446,7 @@ public class FutureDocTest extends AbstractJavaTest {
} }
}, ec); }, ec);
resultFuture.onSuccess(new PrintResult<Object>(), system.dispatcher()); resultFuture.onComplete(new PrintResult<Try<Object>>(), system.dispatcher());
//#reduce //#reduce
Timeout timeout = Timeout.create(Duration.ofSeconds(5)); Timeout timeout = Timeout.create(Duration.ofSeconds(5));
Object result = Await.result(resultFuture, timeout.duration()); Object result = Await.result(resultFuture, timeout.duration());
@ -545,7 +546,7 @@ public class FutureDocTest extends AbstractJavaTest {
} }
}, ec); }, ec);
future.onSuccess(new PrintResult<Integer>(), system.dispatcher()); future.onComplete(new PrintResult<Try<Integer>>(), system.dispatcher());
//#recover //#recover
Timeout timeout = Timeout.create(Duration.ofSeconds(5)); Timeout timeout = Timeout.create(Duration.ofSeconds(5));
int result = Await.result(future, timeout.duration()); int result = Await.result(future, timeout.duration());
@ -574,7 +575,7 @@ public class FutureDocTest extends AbstractJavaTest {
} }
}, ec); }, ec);
future.onSuccess(new PrintResult<Integer>(), system.dispatcher()); future.onComplete(new PrintResult<Try<Integer>>(), system.dispatcher());
//#try-recover //#try-recover
Timeout timeout = Timeout.create(Duration.ofSeconds(5)); Timeout timeout = Timeout.create(Duration.ofSeconds(5));
int result = Await.result(future, timeout.duration()); int result = Await.result(future, timeout.duration());
@ -582,42 +583,10 @@ public class FutureDocTest extends AbstractJavaTest {
} }
@Test @Test
public void useOnSuccessOnFailureAndOnComplete() throws Exception { public void useOnOnComplete() throws Exception {
{ {
Future<String> future = Futures.successful("foo"); Future<String> future = Futures.successful("foo");
//#onSuccess
final ExecutionContext ec = system.dispatcher();
future.onSuccess(new OnSuccess<String>() {
public void onSuccess(String result) {
if ("bar" == result) {
//Do something if it resulted in "bar"
} else {
//Do something if it was some other String
}
}
}, ec);
//#onSuccess
}
{
Future<String> future = Futures.failed(new IllegalStateException("OHNOES"));
//#onFailure
final ExecutionContext ec = system.dispatcher();
future.onFailure(new OnFailure() {
public void onFailure(Throwable failure) {
if (failure instanceof IllegalStateException) {
//Do something if it was this particular failure
} else {
//Do something if it was some other failure
}
}
}, ec);
//#onFailure
}
{
Future<String> future = Futures.successful("foo");
//#onComplete //#onComplete
final ExecutionContext ec = system.dispatcher(); final ExecutionContext ec = system.dispatcher();
@ -648,7 +617,7 @@ public class FutureDocTest extends AbstractJavaTest {
} }
}, ec); }, ec);
future3.onSuccess(new PrintResult<String>(), system.dispatcher()); future3.onComplete(new PrintResult<Try<String>>(), system.dispatcher());
//#zip //#zip
Timeout timeout = Timeout.create(Duration.ofSeconds(5)); Timeout timeout = Timeout.create(Duration.ofSeconds(5));
String result = Await.result(future3, timeout.duration()); String result = Await.result(future3, timeout.duration());
@ -662,7 +631,7 @@ public class FutureDocTest extends AbstractJavaTest {
Future<String> future3 = Futures.successful("bar"); Future<String> future3 = Futures.successful("bar");
// Will have "bar" in this case // Will have "bar" in this case
Future<String> future4 = future1.fallbackTo(future2).fallbackTo(future3); Future<String> future4 = future1.fallbackTo(future2).fallbackTo(future3);
future4.onSuccess(new PrintResult<String>(), system.dispatcher()); future4.onComplete(new PrintResult<Try<String>>(), system.dispatcher());
//#fallback-to //#fallback-to
Timeout timeout = Timeout.create(Duration.ofSeconds(5)); Timeout timeout = Timeout.create(Duration.ofSeconds(5));
String result = Await.result(future4, timeout.duration()); String result = Await.result(future4, timeout.duration());

View file

@ -197,7 +197,7 @@ public class PersistenceQueryDocTest {
public akka.stream.scaladsl.Source<RichEvent, QueryMetadata> byTagsWithMeta( public akka.stream.scaladsl.Source<RichEvent, QueryMetadata> byTagsWithMeta(
scala.collection.Set<String> tags) { scala.collection.Set<String> tags) {
Set<String> jTags = scala.collection.JavaConversions.setAsJavaSet(tags); Set<String> jTags = scala.collection.JavaConverters.setAsJavaSetConverter(tags).asJava();
return javadslReadJournal.byTagsWithMeta(jTags).asScala(); return javadslReadJournal.byTagsWithMeta(jTags).asScala();
} }

View file

@ -5,6 +5,7 @@
package scala.docs.cluster package scala.docs.cluster
import language.postfixOps import language.postfixOps
import scala.util.Success
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor.Actor import akka.actor.Actor
import akka.actor.ActorRef import akka.actor.ActorRef
@ -55,9 +56,8 @@ object TransformationFrontend {
import system.dispatcher import system.dispatcher
system.scheduler.schedule(2.seconds, 2.seconds) { system.scheduler.schedule(2.seconds, 2.seconds) {
implicit val timeout = Timeout(5 seconds) implicit val timeout = Timeout(5 seconds)
(frontend ? TransformationJob("hello-" + counter.incrementAndGet())) onSuccess { (frontend ? TransformationJob("hello-" + counter.incrementAndGet()))
case result println(result) .foreach { result println(result) }
}
} }
} }

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