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:
parent
3bff646218
commit
d274e039f9
141 changed files with 596 additions and 468 deletions
|
|
@ -10,6 +10,7 @@ import akka.actor.ActorSystem;
|
|||
import akka.japi.*;
|
||||
import org.junit.ClassRule;
|
||||
import org.scalatest.junit.JUnitSuite;
|
||||
import scala.Function1;
|
||||
import scala.concurrent.Await;
|
||||
import scala.concurrent.Future;
|
||||
import scala.concurrent.Promise;
|
||||
|
|
@ -26,6 +27,7 @@ import java.util.concurrent.TimeUnit;
|
|||
import static akka.japi.Util.classTag;
|
||||
|
||||
import akka.testkit.AkkaSpec;
|
||||
import scala.util.Try;
|
||||
|
||||
public class JavaFutureTests extends JUnitSuite {
|
||||
|
||||
|
|
@ -59,9 +61,9 @@ public class JavaFutureTests extends JUnitSuite {
|
|||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
Promise<String> cf = Futures.promise();
|
||||
Future<String> f = cf.future();
|
||||
f.onSuccess(new OnSuccess<String>() {
|
||||
public void onSuccess(String result) {
|
||||
if (result.equals("foo"))
|
||||
f.onComplete(new OnComplete<String>() {
|
||||
public void onComplete(Throwable t, String r) {
|
||||
if ("foo".equals(r))
|
||||
latch.countDown();
|
||||
}
|
||||
}, system.dispatcher());
|
||||
|
|
@ -76,8 +78,9 @@ public class JavaFutureTests extends JUnitSuite {
|
|||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
Promise<String> cf = Futures.promise();
|
||||
Future<String> f = cf.future();
|
||||
f.onFailure(new OnFailure() {
|
||||
public void onFailure(Throwable t) {
|
||||
f.onComplete(new OnComplete<String>() {
|
||||
public void onComplete(Throwable t, String r) {
|
||||
// 'null instanceof ...' is always false
|
||||
if (t instanceof NullPointerException)
|
||||
latch.countDown();
|
||||
}
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ import language.postfixOps
|
|||
import java.io.Closeable
|
||||
import java.util.concurrent._
|
||||
import atomic.{ AtomicReference, AtomicInteger }
|
||||
import scala.concurrent.{ future, Await, ExecutionContext }
|
||||
import scala.concurrent.{ Future, Await, ExecutionContext }
|
||||
import scala.concurrent.duration._
|
||||
import java.util.concurrent.ThreadLocalRandom
|
||||
import scala.util.Try
|
||||
|
|
@ -378,7 +378,7 @@ class LightArrayRevolverSchedulerSpec extends AkkaSpec(SchedulerSpec.testConfRev
|
|||
import driver._
|
||||
import system.dispatcher
|
||||
val counter = new AtomicInteger
|
||||
val terminated = future {
|
||||
val terminated = Future {
|
||||
var rounds = 0
|
||||
while (Try(sched.scheduleOnce(Duration.Zero)(())(localEC)).isSuccess) {
|
||||
Thread.sleep(1)
|
||||
|
|
@ -511,7 +511,7 @@ class LightArrayRevolverSchedulerSpec extends AkkaSpec(SchedulerSpec.testConfRev
|
|||
withScheduler() { (sched, driver) ⇒
|
||||
import system.dispatcher
|
||||
val counter = new AtomicInteger
|
||||
future { Thread.sleep(5); driver.close(); sched.close() }
|
||||
Future { Thread.sleep(5); driver.close(); sched.close() }
|
||||
val headroom = 200
|
||||
var overrun = headroom
|
||||
val cap = 1000000
|
||||
|
|
|
|||
|
|
@ -6,7 +6,8 @@ package akka.dispatch
|
|||
|
||||
import java.util.concurrent.{ ExecutorService, Executor, Executors }
|
||||
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 akka.testkit.{ TestLatch, AkkaSpec, DefaultTimeout }
|
||||
import akka.util.SerializedSuspendableExecutionContext
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@
|
|||
package akka.io
|
||||
|
||||
import akka.testkit.{ TestProbe, AkkaSpec }
|
||||
import akka.testkit.SocketUtil._
|
||||
import akka.testkit.SocketUtil.temporaryServerAddresses
|
||||
import Tcp._
|
||||
|
||||
class CapacityLimitSpec extends AkkaSpec("""
|
||||
|
|
|
|||
|
|
@ -24,7 +24,7 @@ import akka.io.Inet.SocketOption
|
|||
import akka.actor._
|
||||
import akka.testkit.{ AkkaSpec, EventFilter, SocketUtil, TestActorRef, TestProbe }
|
||||
import akka.util.{ ByteString, Helpers }
|
||||
import akka.testkit.SocketUtil._
|
||||
import akka.testkit.SocketUtil.temporaryServerAddress
|
||||
import java.util.Random
|
||||
import java.net.SocketTimeoutException
|
||||
import java.nio.file.Files
|
||||
|
|
|
|||
|
|
@ -9,11 +9,10 @@ import scala.collection.immutable
|
|||
import akka.testkit.{ AkkaSpec, TestProbe }
|
||||
import akka.actor.ActorRef
|
||||
import akka.io.Inet.SocketOption
|
||||
import akka.testkit.SocketUtil._
|
||||
import akka.testkit.SocketUtil.temporaryServerAddress
|
||||
import Tcp._
|
||||
import akka.actor.ActorSystem
|
||||
import akka.dispatch.ExecutionContexts
|
||||
import scala.collection.immutable
|
||||
|
||||
trait TcpIntegrationSpecSupport { _: AkkaSpec ⇒
|
||||
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ import java.net.InetSocketAddress
|
|||
import akka.testkit.{ TestProbe, ImplicitSender, AkkaSpec }
|
||||
import akka.util.ByteString
|
||||
import akka.actor.ActorRef
|
||||
import akka.testkit.SocketUtil._
|
||||
import akka.testkit.SocketUtil.temporaryServerAddresses
|
||||
|
||||
class UdpConnectedIntegrationSpec extends AkkaSpec("""
|
||||
akka.loglevel = INFO
|
||||
|
|
|
|||
|
|
@ -11,7 +11,7 @@ import akka.util.ByteString
|
|||
import akka.actor.ActorRef
|
||||
import akka.io.Udp._
|
||||
import akka.io.Inet._
|
||||
import akka.testkit.SocketUtil._
|
||||
import akka.testkit.SocketUtil.temporaryServerAddresses
|
||||
import java.net.DatagramSocket
|
||||
|
||||
class UdpIntegrationSpec extends AkkaSpec("""
|
||||
|
|
|
|||
|
|
@ -53,7 +53,7 @@ class RoundRobinSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
|
|||
val doneLatch = new TestLatch(connectionCount)
|
||||
|
||||
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 {
|
||||
lazy val id = counter.getAndIncrement()
|
||||
|
|
@ -65,7 +65,7 @@ class RoundRobinSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
|
|||
|
||||
for (_ ← 1 to iterationCount; _ ← 1 to connectionCount) {
|
||||
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)
|
||||
|
|
@ -123,7 +123,7 @@ class RoundRobinSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
|
|||
val iterationCount = 10
|
||||
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 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) {
|
||||
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")
|
||||
|
|
@ -154,7 +154,7 @@ class RoundRobinSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
|
|||
val connectionCount = 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 {
|
||||
var n = 0
|
||||
|
|
@ -185,7 +185,7 @@ class RoundRobinSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
|
|||
|
||||
for (_ ← 1 to iterationCount; _ ← 1 to connectionCount) {
|
||||
val id = Await.result((actor ? "hit").mapTo[String], timeout.duration)
|
||||
replies = replies + (id → (replies(id) + 1))
|
||||
replies += (id → (replies(id) + 1))
|
||||
}
|
||||
|
||||
watch(actor)
|
||||
|
|
|
|||
|
|
@ -136,7 +136,7 @@ class BoundedBlockingQueueSpec
|
|||
}
|
||||
|
||||
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"))
|
||||
}
|
||||
}
|
||||
|
|
@ -196,7 +196,7 @@ class BoundedBlockingQueueSpec
|
|||
|
||||
// `f` should still block since the queue is still empty
|
||||
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)
|
||||
}
|
||||
}
|
||||
|
|
@ -297,7 +297,7 @@ class BoundedBlockingQueueSpec
|
|||
|
||||
// `f` should still block since the queue is still empty
|
||||
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"))
|
||||
}
|
||||
}
|
||||
|
|
@ -706,12 +706,12 @@ trait QueueSetupHelper {
|
|||
|
||||
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`
|
||||
*/
|
||||
class TestBackingQueue(events: mutable.MutableList[QueueEvent])
|
||||
class TestBackingQueue(events: mutable.Buffer[QueueEvent])
|
||||
extends util.LinkedList[String] {
|
||||
|
||||
override def poll(): String = {
|
||||
|
|
@ -733,7 +733,7 @@ trait QueueSetupHelper {
|
|||
/**
|
||||
* 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 {
|
||||
|
||||
case class Manual(waitTime: Long = 0, waitingThread: Option[Thread] = None)
|
||||
|
|
@ -798,7 +798,7 @@ trait QueueSetupHelper {
|
|||
}
|
||||
|
||||
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 wrappedNotEmpty = new TestCondition(events, realLock.newCondition(), signalNotEmpty, awaitNotEmpty)
|
||||
|
|
|
|||
|
|
@ -762,11 +762,16 @@ class ByteStringSpec extends WordSpec with Matchers with Checkers {
|
|||
|
||||
"serialize correctly" when {
|
||||
"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 str = hexFromSer(bs)
|
||||
|
||||
require(oldSerd == str)
|
||||
str should be(oldSerd)
|
||||
}
|
||||
|
||||
"given all types of ByteString" in {
|
||||
|
|
|
|||
|
|
@ -11,7 +11,7 @@ import scala.util.Properties
|
|||
|
||||
class LineNumberSpec extends AkkaSpec {
|
||||
|
||||
private val isScala212 = Properties.versionNumberString.startsWith("2.12")
|
||||
private val isScala211 = Properties.versionNumberString.startsWith("2.11")
|
||||
|
||||
"LineNumbers" when {
|
||||
|
||||
|
|
@ -24,11 +24,11 @@ class LineNumberSpec extends AkkaSpec {
|
|||
|
||||
"work for larger functions" in {
|
||||
val result = LineNumbers(twoline)
|
||||
if (isScala212)
|
||||
// because how scala 2.12 does the same as Java Lambdas
|
||||
result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 15, 15))
|
||||
else
|
||||
if (isScala211)
|
||||
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 {
|
||||
|
|
@ -37,11 +37,11 @@ class LineNumberSpec extends AkkaSpec {
|
|||
|
||||
"work for `def`" in {
|
||||
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
|
||||
result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 26, 27))
|
||||
else
|
||||
result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 25, 27))
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -250,7 +250,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit(
|
|||
private def nextName(prefix: String = "a"): String = s"$prefix-${nameCounter.next()}"
|
||||
|
||||
// FIXME eventfilter support in typed testkit
|
||||
import scaladsl.adapter._
|
||||
import akka.actor.typed.scaladsl.adapter._
|
||||
|
||||
implicit val untypedSystem = system.toUntyped
|
||||
|
||||
|
|
@ -748,7 +748,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit(
|
|||
val beh = supervise[String](setup(context ⇒
|
||||
supervise[String](setup { context ⇒
|
||||
probe.ref ! Started
|
||||
scaladsl.Behaviors.empty[String]
|
||||
Behaviors.empty[String]
|
||||
}).onFailure[RuntimeException](strategy)
|
||||
)).onFailure[Exception](strategy)
|
||||
|
||||
|
|
|
|||
|
|
@ -32,18 +32,20 @@ object ReceptionistApiSpec {
|
|||
// to work
|
||||
val registered: Future[Receptionist.Registered] =
|
||||
system.receptionist ? (Receptionist.Register(key, service, _))
|
||||
registered.onSuccess {
|
||||
registered.foreach {
|
||||
case key.Registered(ref) ⇒
|
||||
// ref is the right type here
|
||||
ref ! "woho"
|
||||
case _ ⇒ ()
|
||||
}
|
||||
|
||||
// one-off ask outside of actor, should be uncommon but not rare
|
||||
val found: Future[Receptionist.Listing] =
|
||||
system.receptionist ? (Receptionist.Find(key, _))
|
||||
found.onSuccess {
|
||||
found.foreach {
|
||||
case key.Listing(instances) ⇒
|
||||
instances.foreach(_ ! "woho")
|
||||
case _ ⇒ ()
|
||||
}
|
||||
|
||||
Behaviors.setup[Any] { context ⇒
|
||||
|
|
|
|||
|
|
@ -6,7 +6,7 @@ package akka.actor.typed
|
|||
package scaladsl
|
||||
|
||||
import akka.actor.ExtendedActorSystem
|
||||
import akka.actor.typed.internal.adapter.{ PropsAdapter => _, _ }
|
||||
import akka.actor.typed.internal.adapter.{ PropsAdapter ⇒ _, _ }
|
||||
import akka.annotation.InternalApi
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -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.finalStopMessage")
|
||||
|
||||
# Scala 2.13.0-M5
|
||||
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.actor.dungeon.ChildrenContainer#ChildrenIterable.this")
|
||||
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.actor.dungeon.ChildrenContainer#ChildRestartsIterable.this")
|
||||
|
|
|
|||
|
|
@ -4,4 +4,8 @@ ProblemFilters.exclude[MissingClassProblem]("akka.io.dns.internal.AsyncDnsResolv
|
|||
ProblemFilters.exclude[MissingClassProblem]("akka.io.dns.internal.AsyncDnsResolver$SrvType$")
|
||||
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$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")
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -7,7 +7,7 @@ package akka.compat
|
|||
import akka.annotation.InternalApi
|
||||
import scala.concurrent.{ ExecutionContext, Future ⇒ SFuture }
|
||||
import scala.collection.immutable
|
||||
import scala.collection.compat._
|
||||
import akka.util.ccompat._
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -12,7 +12,7 @@ import scala.annotation.tailrec
|
|||
import scala.collection.LinearSeq
|
||||
import scala.collection.mutable.ListBuffer
|
||||
import scala.reflect.ClassTag
|
||||
import scala.collection.compat._
|
||||
import akka.util.ccompat._
|
||||
|
||||
object ByteIterator {
|
||||
object ByteArrayIterator {
|
||||
|
|
@ -234,7 +234,7 @@ object ByteIterator {
|
|||
}
|
||||
|
||||
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)
|
||||
}
|
||||
|
||||
|
|
@ -391,7 +391,6 @@ object ByteIterator {
|
|||
/**
|
||||
* An iterator over a ByteString.
|
||||
*/
|
||||
|
||||
abstract class ByteIterator extends BufferedIterator[Byte] {
|
||||
def len: Int
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
@ -11,14 +11,13 @@ import java.nio.{ ByteBuffer, ByteOrder }
|
|||
import java.lang.{ Iterable ⇒ JIterable }
|
||||
|
||||
import scala.annotation.{ tailrec, varargs }
|
||||
import scala.collection.StrictOptimizedSeqOps
|
||||
import scala.collection.compat._
|
||||
import scala.collection.mutable.{ Builder, WrappedArray }
|
||||
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.reflect.ClassTag
|
||||
import java.nio.charset.{ Charset, StandardCharsets }
|
||||
import akka.util.ccompat._
|
||||
|
||||
object ByteString {
|
||||
|
||||
|
|
@ -669,6 +668,10 @@ sealed abstract class ByteString
|
|||
override protected def fromSpecific(coll: IterableOnce[Byte]): ByteString = ByteString(coll)
|
||||
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
|
||||
private[akka] def byteStringCompanion: ByteString.Companion
|
||||
// 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))
|
||||
|
||||
def map[A](f: Byte ⇒ Byte): ByteString = {
|
||||
???
|
||||
}
|
||||
def map[A](f: Byte ⇒ Byte): ByteString = fromSpecific(super.map(f))
|
||||
}
|
||||
|
||||
object CompactByteString {
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
@ -7,5 +7,4 @@ package akka.util.ccompat
|
|||
import scala.collection.immutable
|
||||
|
||||
package object imm {
|
||||
type MapLike[K, +V, +This <: immutable.Map[K, V]] = immutable.Map[K, V]
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
@ -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
|
||||
}
|
||||
}
|
||||
|
|
@ -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
|
||||
|
||||
import scala.collection.immutable
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
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
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -22,7 +22,7 @@ import akka.util.{ Helpers, JavaDurationConverters, Timeout }
|
|||
import akka.dispatch.ExecutionContexts
|
||||
|
||||
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,
|
||||
|
|
@ -209,7 +209,7 @@ object ActorSelection {
|
|||
if ((x.indexOf('?') != -1) || (x.indexOf('*') != -1)) SelectChildPattern(x)
|
||||
else if (x == "..") SelectParent
|
||||
else SelectChildName(x)
|
||||
}).to(scala.collection.immutable.IndexedSeq)
|
||||
}).to(immutable.IndexedSeq)
|
||||
new ActorSelection with ScalaActorSelection {
|
||||
override val anchor = anchorRef
|
||||
override val path = compiled
|
||||
|
|
|
|||
|
|
@ -17,7 +17,7 @@ import akka.event.Logging.LogEvent
|
|||
import akka.event.Logging.Error
|
||||
import akka.event.Logging.Warning
|
||||
import scala.util.control.NonFatal
|
||||
import scala.collection.compat._
|
||||
import akka.util.ccompat._
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
|
|||
|
|
@ -8,7 +8,6 @@ import scala.collection.immutable
|
|||
|
||||
import akka.actor.{ InvalidActorNameException, ChildStats, ChildRestartStats, ChildNameReserved, ActorRef }
|
||||
import akka.util.Collections.{ EmptyImmutableSeq, PartialImmutableValuesIterable }
|
||||
import akka.util.ccompat
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -50,13 +49,13 @@ private[akka] object ChildrenContainer {
|
|||
final case class Creation() extends SuspendReason with WaitingForChildren
|
||||
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 isDefinedAt(c: ChildStats) = c.isInstanceOf[ChildRestartStats]
|
||||
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 isDefinedAt(c: ChildStats) = c.isInstanceOf[ChildRestartStats]
|
||||
override final def valuesIterator = stats.valuesIterator
|
||||
|
|
|
|||
|
|
@ -16,7 +16,7 @@ import java.util.function.{ Function ⇒ JFunction }
|
|||
import akka.util.unused
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.collection.compat._
|
||||
import akka.util.ccompat._
|
||||
|
||||
abstract class Dns {
|
||||
|
||||
|
|
@ -59,10 +59,10 @@ object Dns extends ExtensionId[DnsExt] with ExtensionIdProvider {
|
|||
def apply(name: String, addresses: Iterable[InetAddress]): Resolved = {
|
||||
val ipv4: immutable.Seq[Inet4Address] = addresses.iterator.collect({
|
||||
case a: Inet4Address ⇒ a
|
||||
}).to(scala.collection.immutable.IndexedSeq)
|
||||
}).to(immutable.IndexedSeq)
|
||||
val ipv6: immutable.Seq[Inet6Address] = addresses.iterator.collect({
|
||||
case a: Inet6Address ⇒ a
|
||||
}).to(scala.collection.immutable.IndexedSeq)
|
||||
}).to(immutable.IndexedSeq)
|
||||
Resolved(name, ipv4, ipv6)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -21,7 +21,6 @@ import java.lang.{ Iterable ⇒ JIterable }
|
|||
import java.nio.file.Path
|
||||
|
||||
import akka.annotation.InternalApi
|
||||
import scala.collection.immutable
|
||||
|
||||
/**
|
||||
* TCP Extension for Akka’s IO layer.
|
||||
|
|
|
|||
|
|
@ -22,7 +22,6 @@ import scala.annotation.tailrec
|
|||
import scala.collection.immutable
|
||||
import scala.concurrent.duration._
|
||||
import scala.util.control.{ NoStackTrace, NonFatal }
|
||||
import scala.collection.immutable
|
||||
|
||||
/**
|
||||
* Base class for TcpIncomingConnection and TcpOutgoingConnection.
|
||||
|
|
|
|||
|
|
@ -8,7 +8,6 @@ import java.nio.channels.SocketChannel
|
|||
import scala.collection.immutable
|
||||
import akka.actor.ActorRef
|
||||
import akka.io.Inet.SocketOption
|
||||
import scala.collection.immutable
|
||||
|
||||
/**
|
||||
* An actor handling the connection state machine for an incoming, already connected
|
||||
|
|
|
|||
|
|
@ -12,8 +12,7 @@ import akka.io.Inet.{ SoJavaFactories, SocketOption }
|
|||
import akka.util.Helpers.Requiring
|
||||
import akka.util.ByteString
|
||||
import akka.actor._
|
||||
import scala.collection.immutable
|
||||
import scala.collection.compat._
|
||||
import akka.util.ccompat._
|
||||
|
||||
/**
|
||||
* UDP Extension for Akka’s IO layer.
|
||||
|
|
@ -282,7 +281,7 @@ object UdpMessage {
|
|||
* message, or the manager will reply with a [[Udp.CommandFailed]] message.
|
||||
*/
|
||||
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.
|
||||
*/
|
||||
|
|
@ -305,7 +304,7 @@ object UdpMessage {
|
|||
* The “simple sender” will not stop itself, you will have to send it a [[akka.actor.PoisonPill]]
|
||||
* 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.
|
||||
*/
|
||||
|
|
|
|||
|
|
@ -11,8 +11,7 @@ import akka.io.Inet.SocketOption
|
|||
import akka.io.Udp.UdpSettings
|
||||
import akka.util.ByteString
|
||||
import akka.actor._
|
||||
import scala.collection.immutable
|
||||
import scala.collection.compat._
|
||||
import akka.util.ccompat._
|
||||
|
||||
/**
|
||||
* UDP Extension for Akka’s IO layer.
|
||||
|
|
|
|||
|
|
@ -11,7 +11,6 @@ import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
|
|||
import akka.io.Inet.SocketOption
|
||||
import akka.io.Udp._
|
||||
import akka.actor._
|
||||
import scala.collection.immutable
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
|
|||
|
|
@ -19,7 +19,7 @@ import scala.collection.JavaConverters._
|
|||
import scala.collection.immutable
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.util.{ Failure, Success, Try }
|
||||
import scala.collection.compat._
|
||||
import akka.util.ccompat._
|
||||
|
||||
/** INTERNAL API */
|
||||
@InternalApi
|
||||
|
|
@ -39,7 +39,7 @@ private[dns] final class DnsSettings(system: ExtendedActorSystem, c: Config) {
|
|||
parseNameserverAddress(other) :: Nil
|
||||
}
|
||||
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")
|
||||
userAddresses.toList
|
||||
case _ ⇒ throw new IllegalArgumentException("Invalid type for nameservers. Must be a string or string list")
|
||||
|
|
|
|||
|
|
@ -92,10 +92,7 @@ private[io] final class AsyncDnsResolver(
|
|||
|
||||
private def sendQuestion(resolver: ActorRef, message: DnsQuestion): Future[Answer] = {
|
||||
val result = (resolver ? message).mapTo[Answer]
|
||||
result.onComplete {
|
||||
case scala.util.Failure(NonFatal(_)) ⇒ resolver ! DropRequest(message.id)
|
||||
case _ ⇒ ()
|
||||
}
|
||||
result.failed.foreach { _ ⇒ resolver ! DropRequest(message.id) }
|
||||
result
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -21,7 +21,7 @@
|
|||
package akka.routing
|
||||
|
||||
import java.lang.Integer.{ rotateLeft ⇒ rotl }
|
||||
import scala.collection.compat._
|
||||
import akka.util.ccompat._
|
||||
|
||||
/**
|
||||
* An object designed to generate well-distributed non-cryptographic
|
||||
|
|
|
|||
|
|
@ -20,7 +20,7 @@ import akka.dispatch.MessageDispatcher
|
|||
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration._
|
||||
import scala.collection.compat._
|
||||
import akka.util.ccompat._
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -110,7 +110,7 @@ private[akka] class RoutedActorCell(
|
|||
case group: Group ⇒
|
||||
val paths = group.paths(system)
|
||||
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 _ ⇒
|
||||
}
|
||||
preSuperStart()
|
||||
|
|
|
|||
|
|
@ -22,7 +22,7 @@ import java.util.concurrent.atomic.AtomicReference
|
|||
import scala.annotation.tailrec
|
||||
import java.util.NoSuchElementException
|
||||
import akka.annotation.InternalApi
|
||||
import scala.collection.compat._
|
||||
import akka.util.ccompat._
|
||||
|
||||
object Serialization {
|
||||
|
||||
|
|
@ -376,6 +376,8 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
|
|||
system.dynamicAccess.createInstanceFor[Serializer](fqn, List(classOf[ExtendedActorSystem] → system)) recoverWith {
|
||||
case _: NoSuchMethodException ⇒
|
||||
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)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -5,7 +5,7 @@
|
|||
package akka.util
|
||||
|
||||
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`!
|
||||
|
|
|
|||
|
|
@ -95,7 +95,7 @@ class CamelSettings private[camel] (config: Config, dynamicAccess: DynamicAccess
|
|||
val section = config.getConfig("akka.camel.conversions")
|
||||
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)) ⇒
|
||||
m.updated(key, dynamicAccess.getClassFor[AnyRef](fqcn).recover {
|
||||
case e ⇒ throw new ConfigurationException("Could not find/load Camel Converter class [" + fqcn + "]", e)
|
||||
|
|
|
|||
|
|
@ -11,16 +11,16 @@ import akka.AkkaException
|
|||
import scala.reflect.ClassTag
|
||||
import scala.runtime.ScalaRunTime
|
||||
import scala.util.Try
|
||||
import scala.collection.JavaConversions._
|
||||
import akka.dispatch.Mapper
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
/**
|
||||
* 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")
|
||||
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], attachments: JMap[String, DataHandler]) = this(body, headers.toMap, attachments.toMap) //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.asScala.toMap, attachments.asScala.toMap) //Java
|
||||
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)
|
||||
|
|
@ -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 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
|
||||
* message's immutable headers map. Any attempt to modify the returned map will throw an
|
||||
* 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.
|
||||
*/
|
||||
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
|
||||
|
|
@ -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 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
|
||||
* message's immutable headers map. Any attempt to modify the returned map will throw an
|
||||
* 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.
|
||||
*/
|
||||
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>.
|
||||
|
|
@ -250,7 +250,7 @@ object CamelMessage extends ((Any, Map[String, Any]) ⇒ CamelMessage) {
|
|||
* in the Camel message.
|
||||
*/
|
||||
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.
|
||||
|
|
@ -261,7 +261,7 @@ object CamelMessage extends ((Any, Map[String, Any]) ⇒ CamelMessage) {
|
|||
* in the Camel message.
|
||||
*/
|
||||
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
|
||||
|
|
|
|||
|
|
@ -368,7 +368,8 @@ abstract class MixMetricsSelectorBase(selectors: immutable.IndexedSeq[CapacityMe
|
|||
override def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] = {
|
||||
val combined: immutable.IndexedSeq[(Address, Double)] = selectors.flatMap(_.capacity(nodeMetrics).toSeq)
|
||||
// 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)) ⇒
|
||||
val (sum, count) = acc(address)
|
||||
acc + (address → ((sum + capacity, count + 1)))
|
||||
|
|
|
|||
|
|
@ -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]")
|
||||
if (timestamp >= that.timestamp) this // that is older
|
||||
else {
|
||||
// equality is based on the name of the Metric and Set doesn't replace existing element
|
||||
copy(metrics = that.metrics union metrics, timestamp = that.timestamp)
|
||||
// equality is based on the name of the Metric
|
||||
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
|
||||
}
|
||||
// Append metrics missing from either latest or current.
|
||||
// Equality is based on the [[Metric.name]] and [[Set]] doesn't replace existing elements.
|
||||
val merged = updated union latestNode.metrics union currentNode.metrics
|
||||
// Equality is based on the [[Metric.name]]
|
||||
val merged = updated union (latestNode.metrics diff updated) union (currentNode.metrics diff updated diff latestNode.metrics)
|
||||
copy(metrics = merged, timestamp = latestNode.timestamp)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -14,8 +14,10 @@ import akka.cluster.metrics._
|
|||
import akka.serialization.{ BaseSerializer, SerializationExtension, Serializers, SerializerWithStringManifest }
|
||||
import akka.util.ClassLoaderObjectInputStream
|
||||
import akka.protobuf.{ ByteString, MessageLite }
|
||||
import akka.util.ccompat._
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable
|
||||
import scala.collection.JavaConverters.{ asJavaIterableConverter, asScalaBufferConverter, setAsJavaSetConverter }
|
||||
import java.io.NotSerializableException
|
||||
|
||||
|
|
@ -170,9 +172,8 @@ class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithS
|
|||
}
|
||||
|
||||
private def metricsGossipEnvelopeToProto(envelope: MetricsGossipEnvelope): cm.MetricsGossipEnvelope = {
|
||||
import scala.collection.breakOut
|
||||
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 allMetricNames: Vector[String] = allNodeMetrics.foldLeft(Set.empty[String])((s, n) ⇒ s ++ n.metrics.iterator.map(_.name)).toVector
|
||||
val metricNamesMapping = allMetricNames.zipWithIndex.toMap
|
||||
|
|
@ -221,9 +222,8 @@ class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithS
|
|||
metricsGossipEnvelopeFromProto(cm.MetricsGossipEnvelope.parseFrom(decompress(bytes)))
|
||||
|
||||
private def metricsGossipEnvelopeFromProto(envelope: cm.MetricsGossipEnvelope): MetricsGossipEnvelope = {
|
||||
import scala.collection.breakOut
|
||||
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
|
||||
|
||||
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 =
|
||||
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)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -31,6 +31,7 @@ import akka.serialization.SerializerWithStringManifest
|
|||
import akka.util.Timeout
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.scalatest.WordSpecLike
|
||||
import akka.util.ccompat.imm._
|
||||
|
||||
object ClusterShardingSpec {
|
||||
val config = ConfigFactory.parseString(
|
||||
|
|
@ -236,11 +237,11 @@ class ClusterShardingSpec extends ScalaTestWithActorTestKit(ClusterShardingSpec.
|
|||
Cluster(system2).manager ! Join(Cluster(system).selfMember.address)
|
||||
|
||||
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)
|
||||
}
|
||||
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)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -10,7 +10,7 @@ import java.util.zip.GZIPOutputStream
|
|||
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.collection.breakOut
|
||||
import scala.collection.immutable
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.ExtendedActorSystem
|
||||
import akka.cluster.sharding.Shard
|
||||
|
|
@ -20,6 +20,7 @@ import akka.serialization.BaseSerializer
|
|||
import akka.serialization.Serialization
|
||||
import akka.serialization.SerializerWithStringManifest
|
||||
import akka.protobuf.MessageLite
|
||||
import akka.util.ccompat._
|
||||
import java.io.NotSerializableException
|
||||
|
||||
import akka.cluster.sharding.ShardRegion._
|
||||
|
|
@ -214,16 +215,16 @@ private[akka] class ClusterShardingMessageSerializer(val system: ExtendedActorSy
|
|||
|
||||
private def coordinatorStateFromProto(state: sm.CoordinatorState): State = {
|
||||
val shards: Map[String, ActorRef] =
|
||||
state.getShardsList.asScala.toVector.map { entry ⇒
|
||||
state.getShardsList.asScala.toVector.iterator.map { entry ⇒
|
||||
entry.getShardId → resolveActorRef(entry.getRegionRef)
|
||||
}(breakOut)
|
||||
}.toMap
|
||||
|
||||
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]] =
|
||||
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
|
||||
|
||||
State(shards, regions, proxies, unallocatedShards)
|
||||
|
|
@ -299,7 +300,7 @@ private[akka] class ClusterShardingMessageSerializer(val system: ExtendedActorSy
|
|||
|
||||
private def shardRegionStatsFromBinary(bytes: Array[Byte]): ShardRegionStats = {
|
||||
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)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -22,6 +22,7 @@ import akka.remote.testkit.MultiNodeSpec
|
|||
import akka.remote.testkit.STMultiNodeSpec
|
||||
import akka.remote.transport.ThrottlerTransportAdapter.Direction
|
||||
import akka.testkit._
|
||||
import akka.util.ccompat.imm._
|
||||
|
||||
object ClusterShardingFailureSpec {
|
||||
case class Get(id: String)
|
||||
|
|
@ -125,8 +126,8 @@ abstract class ClusterShardingFailureSpec(config: ClusterShardingFailureSpecConf
|
|||
|
||||
within(remaining) {
|
||||
awaitAssert {
|
||||
cluster.state.members.map(_.uniqueAddress) should contain(cluster.selfUniqueAddress)
|
||||
cluster.state.members.map(_.status) should ===(Set(MemberStatus.Up))
|
||||
cluster.state.members.unsorted.map(_.uniqueAddress) should contain(cluster.selfUniqueAddress)
|
||||
cluster.state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -19,6 +19,7 @@ import org.apache.commons.io.FileUtils
|
|||
import scala.concurrent.duration._
|
||||
import akka.cluster.sharding.ShardRegion.GetClusterShardingStats
|
||||
import akka.cluster.sharding.ShardRegion.ClusterShardingStats
|
||||
import akka.util.ccompat.imm._
|
||||
|
||||
object ClusterShardingMinMembersSpec {
|
||||
case object StopEntity
|
||||
|
|
@ -154,7 +155,7 @@ abstract class ClusterShardingMinMembersSpec(config: ClusterShardingMinMembersSp
|
|||
within(remaining) {
|
||||
awaitAssert {
|
||||
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")
|
||||
|
|
|
|||
|
|
@ -15,6 +15,7 @@ import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec }
|
|||
import akka.testkit._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import org.apache.commons.io.FileUtils
|
||||
import akka.util.ccompat.imm._
|
||||
|
||||
import scala.concurrent.duration._
|
||||
|
||||
|
|
@ -175,7 +176,7 @@ abstract class ClusterShardingRememberEntitiesSpec(config: ClusterShardingRememb
|
|||
within(remaining) {
|
||||
awaitAssert {
|
||||
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))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -18,6 +18,7 @@ import akka.remote.testkit.MultiNodeSpec
|
|||
import akka.remote.testkit.STMultiNodeSpec
|
||||
import akka.testkit._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.util.ccompat.imm._
|
||||
|
||||
object MultiDcClusterShardingSpec {
|
||||
sealed trait EntityMsg {
|
||||
|
|
@ -131,7 +132,7 @@ abstract class MultiDcClusterShardingSpec extends MultiNodeSpec(MultiDcClusterSh
|
|||
awaitAssert({
|
||||
withClue(s"Members: ${Cluster(system).state}") {
|
||||
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)
|
||||
|
||||
|
|
|
|||
|
|
@ -16,6 +16,7 @@ import akka.testkit.AkkaSpec
|
|||
import akka.testkit.TestActors.EchoActor
|
||||
import akka.testkit.TestProbe
|
||||
import akka.testkit.WithLogCapturing
|
||||
import akka.util.ccompat.imm._
|
||||
|
||||
object CoordinatedShutdownShardingSpec {
|
||||
val config =
|
||||
|
|
@ -95,9 +96,9 @@ class CoordinatedShutdownShardingSpec extends AkkaSpec(CoordinatedShutdownShardi
|
|||
within(10.seconds) {
|
||||
awaitAssert {
|
||||
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.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) {
|
||||
awaitAssert {
|
||||
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.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.map(_.status) should ===(Set(MemberStatus.Up))
|
||||
Cluster(sys3).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -38,6 +38,7 @@ import com.typesafe.config.Config
|
|||
import akka.remote.DeadlineFailureDetector
|
||||
import akka.dispatch.Dispatchers
|
||||
import akka.util.MessageBuffer
|
||||
import akka.util.ccompat._
|
||||
import scala.collection.immutable.{ HashMap, HashSet }
|
||||
|
||||
object ClusterClientSettings {
|
||||
|
|
@ -332,7 +333,7 @@ final class ClusterClient(settings: ClusterClientSettings) extends Actor with Ac
|
|||
val failureDetector = new DeadlineFailureDetector(acceptableHeartbeatPause, heartbeatInterval)
|
||||
|
||||
var contactPaths: HashSet[ActorPath] =
|
||||
initialContacts.to[HashSet]
|
||||
initialContacts.to(HashSet)
|
||||
val initialContactsSel =
|
||||
contactPaths.map(context.actorSelection)
|
||||
var contacts = initialContactsSel
|
||||
|
|
@ -373,7 +374,7 @@ final class ClusterClient(settings: ClusterClientSettings) extends Actor with Ac
|
|||
{
|
||||
case Contacts(contactPoints) ⇒
|
||||
if (contactPoints.nonEmpty) {
|
||||
contactPaths = contactPoints.map(ActorPath.fromString).to[HashSet]
|
||||
contactPaths = contactPoints.map(ActorPath.fromString).to(HashSet)
|
||||
contacts = contactPaths.map(context.actorSelection)
|
||||
contacts foreach { _ ! Identify(Array.emptyByteArray) }
|
||||
}
|
||||
|
|
@ -423,7 +424,7 @@ final class ClusterClient(settings: ClusterClientSettings) extends Actor with Ac
|
|||
case Contacts(contactPoints) ⇒
|
||||
// refresh of contacts
|
||||
if (contactPoints.nonEmpty) {
|
||||
contactPaths = contactPoints.map(ActorPath.fromString).to[HashSet]
|
||||
contactPaths = contactPoints.map(ActorPath.fromString).to(HashSet)
|
||||
contacts = contactPaths.map(context.actorSelection)
|
||||
}
|
||||
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
|
||||
// load balances the client connections among the nodes in the cluster.
|
||||
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)
|
||||
log.debug("Client [{}] gets contactPoints [{}] (all nodes)", sender().path, contacts.contactPoints.mkString(","))
|
||||
sender() ! contacts
|
||||
|
|
@ -960,11 +961,11 @@ final class ClusterReceptionist(pubSubMediator: ActorRef, settings: ClusterRecep
|
|||
// toStringWithAddress will use the remote address of the client
|
||||
val a = consistentHash.nodeFor(sender().path.toStringWithAddress(cluster.selfAddress))
|
||||
val slice = {
|
||||
val first = nodes.from(a).tail.take(numberOfContacts)
|
||||
val first = nodes.rangeFrom(a).tail.take(numberOfContacts)
|
||||
if (first.size == numberOfContacts) first
|
||||
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)
|
||||
log.debug("Client [{}] gets contactPoints [{}]", sender().path, contacts.contactPoints.mkString(","))
|
||||
sender() ! contacts
|
||||
|
|
@ -992,7 +993,7 @@ final class ClusterReceptionist(pubSubMediator: ActorRef, settings: ClusterRecep
|
|||
|
||||
case SubscribeClusterClients ⇒
|
||||
val subscriber = sender()
|
||||
subscriber ! ClusterClients(clientInteractions.keySet.to[HashSet])
|
||||
subscriber ! ClusterClients(clientInteractions.keySet.to(HashSet))
|
||||
subscribers :+= subscriber
|
||||
context.watch(subscriber)
|
||||
|
||||
|
|
@ -1004,7 +1005,7 @@ final class ClusterReceptionist(pubSubMediator: ActorRef, settings: ClusterRecep
|
|||
self.tell(UnsubscribeClusterClients, subscriber)
|
||||
|
||||
case GetClusterClients ⇒
|
||||
sender() ! ClusterClients(clientInteractions.keySet.to[HashSet])
|
||||
sender() ! ClusterClients(clientInteractions.keySet.to(HashSet))
|
||||
|
||||
case CheckDeadlines ⇒
|
||||
clientInteractions = clientInteractions.filter {
|
||||
|
|
@ -1025,11 +1026,11 @@ final class ClusterReceptionist(pubSubMediator: ActorRef, settings: ClusterRecep
|
|||
log.debug("Received new contact from [{}]", client.path)
|
||||
val clusterClientUp = ClusterClientUp(client)
|
||||
subscribers.foreach(_ ! clusterClientUp)
|
||||
clientsPublished = clientInteractions.keySet.to[HashSet]
|
||||
clientsPublished = clientInteractions.keySet.to(HashSet)
|
||||
}
|
||||
|
||||
def publishClientsUnreachable(): Unit = {
|
||||
val publishableClients = clientInteractions.keySet.to[HashSet]
|
||||
val publishableClients = clientInteractions.keySet.to(HashSet)
|
||||
for (c ← clientsPublished if !publishableClients.contains(c)) {
|
||||
log.debug("Lost contact with [{}]", c.path)
|
||||
val clusterClientUnreachable = ClusterClientUnreachable(c)
|
||||
|
|
|
|||
|
|
@ -569,8 +569,8 @@ class DistributedPubSubMediator(settings: DistributedPubSubSettings) extends Act
|
|||
case _ ⇒
|
||||
(for {
|
||||
(_, bucket) ← registry
|
||||
valueHolder ← bucket.content.get(path)
|
||||
routee ← valueHolder.routee
|
||||
valueHolder ← bucket.content.get(path).toSeq
|
||||
routee ← valueHolder.routee.toSeq
|
||||
} yield routee).toVector
|
||||
}
|
||||
|
||||
|
|
@ -751,8 +751,8 @@ class DistributedPubSubMediator(settings: DistributedPubSubSettings) extends Act
|
|||
val refs = for {
|
||||
(address, bucket) ← registry
|
||||
if !(allButSelf && address == selfAddress) // if we should skip sender() node and current address == self address => skip
|
||||
valueHolder ← bucket.content.get(path)
|
||||
ref ← valueHolder.ref
|
||||
valueHolder ← bucket.content.get(path).toSeq
|
||||
ref ← valueHolder.ref.toSeq
|
||||
} yield ref
|
||||
if (refs.isEmpty) ignoreOrSendToDeadLetters(msg)
|
||||
else refs.foreach(_.forward(msg))
|
||||
|
|
@ -793,7 +793,7 @@ class DistributedPubSubMediator(settings: DistributedPubSubSettings) extends Act
|
|||
val topicPrefix = self.path.toStringWithoutAddress
|
||||
(for {
|
||||
(_, bucket) ← registry
|
||||
(key, value) ← bucket.content
|
||||
(key, value) ← bucket.content.toSeq
|
||||
if key.startsWith(topicPrefix)
|
||||
topic = key.substring(topicPrefix.length + 1)
|
||||
if !topic.contains('/') // exclude group topics
|
||||
|
|
|
|||
|
|
@ -5,7 +5,6 @@
|
|||
package akka.cluster.pubsub.protobuf
|
||||
|
||||
import akka.serialization._
|
||||
import scala.collection.breakOut
|
||||
import akka.actor.{ Address, ExtendedActorSystem }
|
||||
import java.io.{ ByteArrayInputStream, ByteArrayOutputStream }
|
||||
import akka.protobuf.{ ByteString, MessageLite }
|
||||
|
|
@ -17,6 +16,7 @@ import scala.collection.JavaConverters._
|
|||
import akka.cluster.pubsub.DistributedPubSubMediator._
|
||||
import akka.cluster.pubsub.DistributedPubSubMediator.Internal._
|
||||
import akka.actor.ActorRef
|
||||
import akka.util.ccompat._
|
||||
import scala.collection.immutable.TreeMap
|
||||
import java.io.NotSerializableException
|
||||
|
||||
|
|
@ -127,8 +127,8 @@ private[akka] class DistributedPubSubMessageSerializer(val system: ExtendedActor
|
|||
|
||||
private def statusFromProto(status: dm.Status): Status = {
|
||||
val isReplyToStatus = if (status.hasReplyToStatus) status.getReplyToStatus else false
|
||||
Status(status.getVersionsList.asScala.map(v ⇒
|
||||
addressFromProto(v.getAddress) → v.getTimestamp)(breakOut), isReplyToStatus)
|
||||
Status(status.getVersionsList.asScala.iterator.map(v ⇒
|
||||
addressFromProto(v.getAddress) → v.getTimestamp).toMap, isReplyToStatus)
|
||||
}
|
||||
|
||||
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 =
|
||||
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)
|
||||
}(breakOut)
|
||||
})
|
||||
Bucket(addressFromProto(b.getOwner), b.getVersion, content)
|
||||
})
|
||||
|
||||
|
|
|
|||
|
|
@ -70,6 +70,7 @@ class ClusterSingletonLeavingSpeedSpec extends AkkaSpec("""
|
|||
override def expectedTestDuration: FiniteDuration = 10.minutes
|
||||
|
||||
def join(from: ActorSystem, to: ActorSystem, probe: ActorRef): Unit = {
|
||||
|
||||
from.actorOf(
|
||||
ClusterSingletonManager.props(
|
||||
singletonProps = TheSingleton.props(probe),
|
||||
|
|
@ -79,9 +80,10 @@ class ClusterSingletonLeavingSpeedSpec extends AkkaSpec("""
|
|||
|
||||
Cluster(from).join(Cluster(to).selfAddress)
|
||||
within(15.seconds) {
|
||||
import akka.util.ccompat.imm._
|
||||
awaitAssert {
|
||||
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))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -62,10 +62,11 @@ class ClusterSingletonRestart2Spec extends AkkaSpec("""
|
|||
name = "echo")
|
||||
|
||||
within(45.seconds) {
|
||||
import akka.util.ccompat.imm._
|
||||
awaitAssert {
|
||||
Cluster(from) join Cluster(to).selfAddress
|
||||
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))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -43,10 +43,11 @@ class ClusterSingletonRestartSpec extends AkkaSpec("""
|
|||
name = "echo")
|
||||
|
||||
within(10.seconds) {
|
||||
import akka.util.ccompat.imm._
|
||||
awaitAssert {
|
||||
Cluster(from) join Cluster(to).selfAddress
|
||||
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))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -16,7 +16,7 @@ import akka.dispatch.MonitorableThreadFactory
|
|||
import akka.event.{ Logging, LoggingAdapter }
|
||||
import akka.japi.Util
|
||||
import akka.pattern._
|
||||
import akka.remote.{ DefaultFailureDetectorRegistry, _ }
|
||||
import akka.remote.{ UniqueAddress ⇒ _, _ }
|
||||
import com.typesafe.config.{ Config, ConfigFactory }
|
||||
|
||||
import scala.annotation.varargs
|
||||
|
|
|
|||
|
|
@ -1258,9 +1258,7 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef, joinConfigCompatCh
|
|||
}
|
||||
|
||||
if (changedMembers.nonEmpty) {
|
||||
// replace changed members
|
||||
val newMembers = changedMembers union localMembers
|
||||
val newGossip = localGossip.copy(members = newMembers)
|
||||
val newGossip = localGossip.update(changedMembers)
|
||||
updateLatestGossip(newGossip)
|
||||
|
||||
// log status changes
|
||||
|
|
|
|||
|
|
@ -15,8 +15,8 @@ import akka.event.EventStream
|
|||
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
|
||||
import akka.actor.DeadLetterSuppression
|
||||
import akka.annotation.{ DoNotInherit, InternalApi }
|
||||
import akka.util.ccompat._
|
||||
|
||||
import scala.collection.breakOut
|
||||
import scala.runtime.AbstractFunction5
|
||||
|
||||
/**
|
||||
|
|
@ -162,7 +162,7 @@ object ClusterEvent {
|
|||
/**
|
||||
* 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
|
||||
|
|
@ -380,10 +380,10 @@ object ClusterEvent {
|
|||
else {
|
||||
val newGossip = newState.latestGossip
|
||||
val oldUnreachableNodes = oldState.dcReachabilityNoOutsideNodes.allUnreachableOrTerminated
|
||||
newState.dcReachabilityNoOutsideNodes.allUnreachableOrTerminated.collect {
|
||||
newState.dcReachabilityNoOutsideNodes.allUnreachableOrTerminated.iterator.collect {
|
||||
case node if !oldUnreachableNodes.contains(node) && node != newState.selfUniqueAddress ⇒
|
||||
UnreachableMember(newGossip.member(node))
|
||||
}(collection.breakOut)
|
||||
}.to(immutable.IndexedSeq)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -393,10 +393,10 @@ object ClusterEvent {
|
|||
if (newState eq oldState) Nil
|
||||
else {
|
||||
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 ⇒
|
||||
ReachableMember(newGossip.member(node))
|
||||
}(collection.breakOut)
|
||||
}.to(immutable.IndexedSeq)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -418,7 +418,7 @@ object ClusterEvent {
|
|||
if (newState eq oldState) Nil
|
||||
else {
|
||||
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 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 ⇒
|
||||
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 == WeaklyUp ⇒ MemberWeaklyUp(m)
|
||||
case m if m.status == Up ⇒ MemberUp(m)
|
||||
|
|
@ -462,7 +463,7 @@ object ClusterEvent {
|
|||
}
|
||||
|
||||
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()
|
||||
}
|
||||
|
|
@ -572,8 +573,8 @@ private[cluster] final class ClusterDomainEventPublisher extends Actor with Acto
|
|||
unreachable = unreachable,
|
||||
seenBy = membershipState.latestGossip.seenBy.map(_.address),
|
||||
leader = membershipState.leader.map(_.address),
|
||||
roleLeaderMap = membershipState.latestGossip.allRoles.map(r ⇒
|
||||
r → membershipState.roleLeader(r).map(_.address))(collection.breakOut),
|
||||
roleLeaderMap = membershipState.latestGossip.allRoles.iterator.map(r ⇒
|
||||
r → membershipState.roleLeader(r).map(_.address)).toMap,
|
||||
unreachableDataCenters)
|
||||
receiver ! state
|
||||
}
|
||||
|
|
|
|||
|
|
@ -14,6 +14,7 @@ import akka.cluster.ClusterEvent._
|
|||
import akka.remote.FailureDetectorRegistry
|
||||
import akka.remote.HeartbeatMessage
|
||||
import akka.annotation.InternalApi
|
||||
import akka.util.ccompat._
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
|
|
@ -338,13 +339,13 @@ private[cluster] final case class HeartbeatNodeRing(
|
|||
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 =
|
||||
if (remaining == 0)
|
||||
slice1
|
||||
else {
|
||||
// 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
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -10,8 +10,10 @@ import akka.cluster.ClusterEvent._
|
|||
import akka.cluster.ClusterSettings.DataCenter
|
||||
import akka.remote.FailureDetectorRegistry
|
||||
import akka.util.ConstantFun
|
||||
import akka.util.ccompat._
|
||||
|
||||
import scala.collection.{ SortedSet, breakOut }
|
||||
import scala.collection.SortedSet
|
||||
import scala.collection.immutable
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -270,8 +272,8 @@ private[cluster] final case class CrossDcHeartbeatingState(
|
|||
val allOtherNodes = otherDcs.values
|
||||
|
||||
allOtherNodes.flatMap(
|
||||
_.take(nrOfMonitoredNodesPerDc)
|
||||
.map(_.uniqueAddress)(breakOut)).toSet
|
||||
_.take(nrOfMonitoredNodesPerDc).iterator
|
||||
.map(_.uniqueAddress).to(immutable.IndexedSeq)).toSet
|
||||
}
|
||||
|
||||
/** Lists addresses in diven DataCenter that this node should send heartbeats to */
|
||||
|
|
@ -280,8 +282,8 @@ private[cluster] final case class CrossDcHeartbeatingState(
|
|||
else {
|
||||
val otherNodes = state.getOrElse(dc, emptyMembersSortedSet)
|
||||
otherNodes
|
||||
.take(nrOfMonitoredNodesPerDc)
|
||||
.map(_.uniqueAddress)(breakOut)
|
||||
.take(nrOfMonitoredNodesPerDc).iterator
|
||||
.map(_.uniqueAddress).to(immutable.Set)
|
||||
}
|
||||
|
||||
def allMembers: Iterable[Member] =
|
||||
|
|
|
|||
|
|
@ -87,7 +87,7 @@ private[cluster] final case class Gossip(
|
|||
}
|
||||
|
||||
@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 =
|
||||
if (members.size <= 1) false
|
||||
|
|
@ -209,7 +209,7 @@ private[cluster] final case class Gossip(
|
|||
}
|
||||
|
||||
def update(updatedMembers: immutable.SortedSet[Member]): Gossip = {
|
||||
copy(members = updatedMembers union members)
|
||||
copy(members = updatedMembers union (members diff updatedMembers))
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -8,6 +8,7 @@ import java.util
|
|||
|
||||
import akka.actor.ExtendedActorSystem
|
||||
import akka.annotation.{ DoNotInherit, InternalApi }
|
||||
import akka.util.ccompat._
|
||||
import com.typesafe.config.{ Config, ConfigFactory, ConfigValue }
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
|
@ -48,7 +49,7 @@ object JoinConfigCompatChecker {
|
|||
}
|
||||
|
||||
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
|
||||
else Invalid(incompatibleKeys.to[im.Seq])
|
||||
else Invalid(incompatibleKeys.to(im.Seq))
|
||||
}
|
||||
|
||||
exists(requiredKeys, toCheck) ++ checkEquality
|
||||
|
|
@ -123,7 +124,7 @@ object JoinConfigCompatChecker {
|
|||
*/
|
||||
@InternalApi
|
||||
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)
|
||||
}
|
||||
|
||||
|
|
@ -147,7 +148,7 @@ object JoinConfigCompatChecker {
|
|||
new JoinConfigCompatChecker {
|
||||
override val requiredKeys: im.Seq[String] = {
|
||||
// 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 =
|
||||
checkers.foldLeft(Valid: ConfigValidation) { (acc, checker) ⇒
|
||||
|
|
|
|||
|
|
@ -11,9 +11,9 @@ import scala.collection.SortedSet
|
|||
import akka.cluster.ClusterSettings.DataCenter
|
||||
import akka.cluster.MemberStatus._
|
||||
import akka.annotation.InternalApi
|
||||
import akka.util.ccompat._
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.breakOut
|
||||
import scala.util.Random
|
||||
|
||||
/**
|
||||
|
|
@ -275,10 +275,10 @@ import scala.util.Random
|
|||
if (preferNodesWithDifferentView(state)) {
|
||||
// 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
|
||||
latestGossip.members.collect {
|
||||
latestGossip.members.iterator.collect {
|
||||
case m if m.dataCenter == state.selfDc && !latestGossip.seenByNode(m.uniqueAddress) && state.validNodeForGossip(m.uniqueAddress) ⇒
|
||||
m.uniqueAddress
|
||||
}(breakOut)
|
||||
}.to(Vector)
|
||||
} else Vector.empty
|
||||
|
||||
// Fall back to localGossip
|
||||
|
|
|
|||
|
|
@ -7,7 +7,7 @@ package akka.cluster
|
|||
import akka.annotation.InternalApi
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.collection.breakOut
|
||||
import akka.util.ccompat._
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -72,9 +72,8 @@ private[cluster] class Reachability private (
|
|||
(observerRowsMap, allUnreachable, allTerminated)
|
||||
} else {
|
||||
val mapBuilder = scala.collection.mutable.Map.empty[UniqueAddress, Map[UniqueAddress, Reachability.Record]]
|
||||
import scala.collection.mutable.SetBuilder
|
||||
val terminatedBuilder = new SetBuilder[UniqueAddress, Set[UniqueAddress]](Set.empty)
|
||||
val unreachableBuilder = new SetBuilder[UniqueAddress, Set[UniqueAddress]](Set.empty)
|
||||
var allTerminated = Set.empty[UniqueAddress]
|
||||
var allUnreachable = Set.empty[UniqueAddress]
|
||||
|
||||
records foreach { r ⇒
|
||||
val m = mapBuilder.get(r.observer) match {
|
||||
|
|
@ -83,15 +82,13 @@ private[cluster] class Reachability private (
|
|||
}
|
||||
mapBuilder += (r.observer → m)
|
||||
|
||||
if (r.status == Unreachable) unreachableBuilder += r.subject
|
||||
else if (r.status == Terminated) terminatedBuilder += r.subject
|
||||
if (r.status == Unreachable) allUnreachable += r.subject
|
||||
else if (r.status == Terminated) allTerminated += r.subject
|
||||
}
|
||||
|
||||
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 = {
|
||||
val nodesSet = nodes.to[immutable.HashSet]
|
||||
val nodesSet = nodes.to(immutable.HashSet)
|
||||
val newRecords = records.filterNot(r ⇒ nodesSet(r.observer) || nodesSet(r.subject))
|
||||
val newVersions = versions -- nodes
|
||||
Reachability(newRecords, newVersions)
|
||||
|
|
@ -265,16 +262,16 @@ private[cluster] class Reachability private (
|
|||
observerRows(observer) match {
|
||||
case None ⇒ Set.empty
|
||||
case Some(observerRows) ⇒
|
||||
observerRows.collect {
|
||||
observerRows.iterator.collect {
|
||||
case (subject, record) if record.status == Unreachable ⇒ subject
|
||||
}(breakOut)
|
||||
}.to(immutable.Set)
|
||||
}
|
||||
|
||||
def observersGroupedByUnreachable: Map[UniqueAddress, Set[UniqueAddress]] = {
|
||||
records.groupBy(_.subject).collect {
|
||||
case (subject, records) if records.exists(_.status == Unreachable) ⇒
|
||||
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)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -21,6 +21,8 @@ import akka.annotation.InternalApi
|
|||
import akka.cluster.InternalClusterAction._
|
||||
import akka.cluster.routing.{ ClusterRouterPool, ClusterRouterPoolSettings }
|
||||
import akka.routing.Pool
|
||||
import akka.util.ccompat._
|
||||
import akka.util.ccompat.imm._
|
||||
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 allAddresses: Vector[UniqueAddress] = allMembers.map(_.uniqueAddress) ++ gossip.tombstones.keys
|
||||
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 allHashes = gossip.version.versions.keys.to[Vector]
|
||||
val allHashes = gossip.version.versions.keys.to(Vector)
|
||||
val hashMapping = allHashes.zipWithIndex.toMap
|
||||
|
||||
def mapUniqueAddress(uniqueAddress: UniqueAddress): Integer = mapWithErrorMessage(addressMapping, uniqueAddress, "address")
|
||||
|
|
@ -403,7 +405,7 @@ final class ClusterMessageSerializer(val system: ExtendedActorSystem) extends Se
|
|||
.build()
|
||||
|
||||
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 overview = cm.GossipOverview.newBuilder.addAllSeen(seen.asJava).
|
||||
|
|
@ -447,11 +449,10 @@ final class ClusterMessageSerializer(val system: ExtendedActorSystem) extends Se
|
|||
gossipStatusFromProto(cm.GossipStatus.parseFrom(bytes))
|
||||
|
||||
private def gossipFromProto(gossip: cm.Gossip): Gossip = {
|
||||
import scala.collection.breakOut
|
||||
val addressMapping: Vector[UniqueAddress] =
|
||||
gossip.getAllAddressesList.asScala.map(uniqueAddressFromProto)(breakOut)
|
||||
val roleMapping: Vector[String] = gossip.getAllRolesList.asScala.map(identity)(breakOut)
|
||||
val hashMapping: Vector[String] = gossip.getAllHashesList.asScala.map(identity)(breakOut)
|
||||
gossip.getAllAddressesList.asScala.iterator.map(uniqueAddressFromProto).to(immutable.Vector)
|
||||
val roleMapping: Vector[String] = gossip.getAllRolesList.asScala.iterator.map(identity).to(immutable.Vector)
|
||||
val hashMapping: Vector[String] = gossip.getAllHashesList.asScala.iterator.map(identity).to(immutable.Vector)
|
||||
|
||||
def reachabilityFromProto(observerReachability: Iterable[cm.ObserverReachability]): Reachability = {
|
||||
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) =
|
||||
(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 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 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)
|
||||
}
|
||||
|
||||
private def vectorClockFromProto(version: cm.VectorClock, hashMapping: immutable.Seq[String]) = {
|
||||
import scala.collection.breakOut
|
||||
VectorClock(version.getVersionsList.asScala.map(
|
||||
v ⇒ (VectorClock.Node.fromHash(hashMapping(v.getHashIndex)), v.getTimestamp))(breakOut))
|
||||
VectorClock(scala.collection.immutable.TreeMap.from(version.getVersionsList.asScala.iterator.map(
|
||||
v ⇒ (VectorClock.Node.fromHash(hashMapping(v.getHashIndex)), v.getTimestamp))))
|
||||
}
|
||||
|
||||
private def gossipEnvelopeFromProto(envelope: cm.GossipEnvelope): GossipEnvelope = {
|
||||
|
|
|
|||
|
|
@ -354,11 +354,13 @@ private[akka] class ClusterRouterPoolActor(
|
|||
None
|
||||
} else {
|
||||
// find the node with least routees
|
||||
val numberOfRouteesPerNode: Map[Address, Int] =
|
||||
currentRoutees.foldLeft(currentNodes.map(_ → 0).toMap.withDefaultValue(0)) { (acc, x) ⇒
|
||||
val numberOfRouteesPerNode: Map[Address, Int] = {
|
||||
val nodeMap: Map[Address, Int] = currentNodes.map(_ → 0).toMap.withDefaultValue(0)
|
||||
currentRoutees.foldLeft(nodeMap) { (acc, x) ⇒
|
||||
val address = fullAddress(x)
|
||||
acc + (address → (acc(address) + 1))
|
||||
}
|
||||
}
|
||||
|
||||
val (address, count) = numberOfRouteesPerNode.minBy(_._2)
|
||||
if (count < settings.maxInstancesPerNode) Some(address) else None
|
||||
|
|
|
|||
|
|
@ -10,6 +10,7 @@ import akka.remote.testkit.MultiNodeConfig
|
|||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import akka.cluster.MemberStatus._
|
||||
import akka.util.ccompat.imm._
|
||||
|
||||
object MinMembersBeforeUpMultiJvmSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
|
|
@ -131,12 +132,12 @@ abstract class MinMembersBeforeUpBase(multiNodeConfig: MultiNodeConfig)
|
|||
clusterView.refreshCurrentState()
|
||||
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
|
||||
1 to 5 foreach { _ ⇒
|
||||
Thread.sleep(1000)
|
||||
clusterView.members.map(_.address) should ===(expectedAddresses)
|
||||
clusterView.members.map(_.status) should ===(Set(Joining))
|
||||
clusterView.members.unsorted.map(_.status) should ===(Set(Joining))
|
||||
}
|
||||
}
|
||||
enterBarrier("second-joined")
|
||||
|
|
|
|||
|
|
@ -14,6 +14,7 @@ import com.typesafe.config.ConfigFactory
|
|||
import scala.collection.immutable
|
||||
import scala.collection.immutable.SortedSet
|
||||
import scala.concurrent.duration._
|
||||
import akka.util.ccompat.imm._
|
||||
|
||||
object MultiDcHeartbeatTakingOverSpecMultiJvmSpec extends MultiNodeConfig {
|
||||
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 -----------
|
||||
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 expectedBetaHeartbeaterRoles: SortedSet[RoleName] = SortedSet.empty
|
||||
var expectedBetaHeartbeaterRoles: List[RoleName] = List.empty
|
||||
|
||||
var expectedNoActiveHeartbeatSenderRoles: Set[RoleName] = Set.empty
|
||||
// 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] =
|
||||
membersByAge(dataCenter).take(n)
|
||||
|
||||
private def membersAsRoles(ms: SortedSet[Member]): SortedSet[RoleName] = {
|
||||
val res = ms.flatMap(m ⇒ roleName(m.address))
|
||||
private def membersAsRoles(ms: SortedSet[Member]): List[RoleName] = {
|
||||
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}")
|
||||
res
|
||||
}
|
||||
|
|
|
|||
|
|
@ -17,6 +17,7 @@ import akka.testkit._
|
|||
import akka.testkit.TestEvent._
|
||||
import akka.actor.{ Actor, ActorRef, ActorSystem, Address, Deploy, PoisonPill, Props, RootActorPath }
|
||||
import akka.event.Logging.ErrorLevel
|
||||
import akka.util.ccompat._
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import scala.collection.immutable
|
||||
|
|
@ -24,6 +25,7 @@ import java.util.concurrent.ConcurrentHashMap
|
|||
|
||||
import akka.remote.DefaultFailureDetectorRegistry
|
||||
import akka.cluster.ClusterEvent.{ MemberEvent, MemberRemoved }
|
||||
import akka.util.ccompat.imm._
|
||||
|
||||
import scala.concurrent.Await
|
||||
|
||||
|
|
@ -273,7 +275,7 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec with WatchedByCoro
|
|||
* be determined from the `RoleName`.
|
||||
*/
|
||||
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
|
||||
|
|
@ -310,7 +312,7 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec with WatchedByCoro
|
|||
if (!canNotBePartOfMemberRing.isEmpty) // don't run this on an empty set
|
||||
awaitAssert(canNotBePartOfMemberRing foreach (a ⇒ clusterView.members.map(_.address) should not contain (a)))
|
||||
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
|
||||
val expectedLeader = clusterView.members.collectFirst {
|
||||
case m if m.dataCenter == cluster.settings.SelfDataCenter ⇒ m.address
|
||||
|
|
|
|||
|
|
@ -7,6 +7,7 @@ package akka.cluster
|
|||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import akka.util.ccompat.imm._
|
||||
|
||||
object NodeMembershipMultiJvmSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
|
|
@ -40,7 +41,7 @@ abstract class NodeMembershipSpec
|
|||
cluster.join(first)
|
||||
awaitAssert(clusterView.members.size should ===(2))
|
||||
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")
|
||||
|
|
@ -54,7 +55,7 @@ abstract class NodeMembershipSpec
|
|||
|
||||
awaitAssert(clusterView.members.size should ===(3))
|
||||
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")
|
||||
}
|
||||
|
|
|
|||
|
|
@ -19,6 +19,7 @@ import akka.actor.Actor
|
|||
import akka.actor.RootActorPath
|
||||
import akka.cluster.MemberStatus._
|
||||
import akka.actor.Deploy
|
||||
import akka.util.ccompat.imm._
|
||||
|
||||
object RestartFirstSeedNodeMultiJvmSpec extends MultiNodeConfig {
|
||||
val seed1 = role("seed1")
|
||||
|
|
@ -97,7 +98,7 @@ abstract class RestartFirstSeedNodeSpec
|
|||
runOn(seed1) {
|
||||
Cluster(seed1System).joinSeedNodes(seedNodes)
|
||||
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) {
|
||||
cluster.joinSeedNodes(seedNodes)
|
||||
|
|
@ -116,7 +117,7 @@ abstract class RestartFirstSeedNodeSpec
|
|||
Cluster(restartedSeed1System).joinSeedNodes(seedNodes)
|
||||
within(20.seconds) {
|
||||
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) {
|
||||
|
|
|
|||
|
|
@ -18,6 +18,7 @@ import akka.remote.testkit.MultiNodeConfig
|
|||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.util.ccompat.imm._
|
||||
|
||||
object RestartNode2SpecMultiJvmSpec extends MultiNodeConfig {
|
||||
val seed1 = role("seed1")
|
||||
|
|
@ -99,7 +100,7 @@ abstract class RestartNode2SpecSpec
|
|||
runOn(seed1) {
|
||||
Cluster(seed1System).joinSeedNodes(seedNodes)
|
||||
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) {
|
||||
cluster.joinSeedNodes(seedNodes)
|
||||
|
|
@ -118,7 +119,7 @@ abstract class RestartNode2SpecSpec
|
|||
Cluster(restartedSeed1System).joinSeedNodes(seedNodes)
|
||||
within(30.seconds) {
|
||||
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) {
|
||||
|
|
|
|||
|
|
@ -18,6 +18,7 @@ import akka.remote.testkit.MultiNodeSpec
|
|||
import akka.remote.transport.ThrottlerTransportAdapter.Direction
|
||||
import akka.testkit._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.util.ccompat.imm._
|
||||
|
||||
object RestartNode3MultiJvmSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
|
|
@ -133,7 +134,7 @@ abstract class RestartNode3Spec
|
|||
runOn(second) {
|
||||
Cluster(restartedSecondSystem).joinSeedNodes(seedNodes)
|
||||
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) {
|
||||
awaitAssert {
|
||||
|
|
|
|||
|
|
@ -23,6 +23,7 @@ import akka.remote.testkit.MultiNodeConfig
|
|||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.util.ccompat.imm._
|
||||
|
||||
object RestartNodeMultiJvmSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
|
|
@ -121,7 +122,7 @@ abstract class RestartNodeSpec
|
|||
runOn(second) {
|
||||
Cluster(secondSystem).joinSeedNodes(seedNodes)
|
||||
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")
|
||||
|
||||
|
|
@ -139,7 +140,7 @@ abstract class RestartNodeSpec
|
|||
runOn(second) {
|
||||
Cluster(restartedSecondSystem).joinSeedNodes(seedNodes)
|
||||
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) {
|
||||
awaitAssert {
|
||||
|
|
|
|||
|
|
@ -345,7 +345,7 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig {
|
|||
class PhiObserver extends Actor with ActorLogging {
|
||||
val cluster = Cluster(context.system)
|
||||
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 nodes = Set.empty[Address]
|
||||
|
||||
|
|
|
|||
|
|
@ -19,6 +19,7 @@ import akka.testkit._
|
|||
import akka.actor.Props
|
||||
import akka.cluster.MultiNodeClusterSpec.EndActor
|
||||
import akka.remote.RARP
|
||||
import akka.util.ccompat.imm._
|
||||
|
||||
object UnreachableNodeJoinsAgainMultiNodeConfig extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
|
|
@ -185,7 +186,7 @@ abstract class UnreachableNodeJoinsAgainSpec
|
|||
within(30 seconds) {
|
||||
awaitAssert(Cluster(freshSystem).readView.members.map(_.address) should contain(victimAddress))
|
||||
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
|
||||
|
|
|
|||
|
|
@ -37,7 +37,7 @@ class ClusterDomainEventSpec extends WordSpec with Matchers {
|
|||
val selfDummyAddress = UniqueAddress(Address("akka.tcp", "sys", "selfDummy", 2552), 17L)
|
||||
|
||||
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 =
|
||||
state(g, selfDummyAddress)
|
||||
|
|
|
|||
|
|
@ -18,7 +18,7 @@ class ReachabilityPerfSpec extends WordSpec with Matchers {
|
|||
val node = Address("akka.tcp", "sys", "a", 2552)
|
||||
|
||||
private def createReachabilityOfSize(base: Reachability, size: Int): Reachability =
|
||||
(base /: (1 to size)) {
|
||||
(1 to size).foldLeft(base) {
|
||||
case (r, i) ⇒
|
||||
val observer = UniqueAddress(address.copy(host = Some("node-" + i)), i.toLong)
|
||||
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 = {
|
||||
val observers = base.allObservers.take(count)
|
||||
val subjects = Stream.continually(base.allObservers).flatten.iterator
|
||||
(base /: observers) {
|
||||
observers.foldLeft(base) {
|
||||
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()) }
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -12,14 +12,14 @@ object VectorClockPerfSpec {
|
|||
import VectorClock._
|
||||
|
||||
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) ⇒
|
||||
val node = Node(i.toString)
|
||||
(vc :+ node, nodes + node)
|
||||
}
|
||||
|
||||
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)
|
||||
}
|
||||
vc.copy(versions = versions)
|
||||
|
|
|
|||
|
|
@ -98,9 +98,7 @@ trait JavaLoggingAdapter extends LoggingAdapter {
|
|||
|
||||
if (loggingExecutionContext.isDefined) {
|
||||
implicit val context = loggingExecutionContext.get
|
||||
Future(logger.log(record)).onFailure {
|
||||
case thrown: Throwable ⇒ thrown.printStackTrace()
|
||||
}
|
||||
Future(logger.log(record)).failed.foreach { _.printStackTrace() }
|
||||
} else
|
||||
logger.log(record)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -11,6 +11,7 @@ import akka.annotation.InternalApi
|
|||
import akka.cluster.ddata.Key.KeyId
|
||||
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation
|
||||
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation.NoDeltaPlaceholder
|
||||
import akka.util.ccompat._
|
||||
|
||||
/**
|
||||
* 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] =
|
||||
entries.from(version) match {
|
||||
entries.rangeFrom(version) match {
|
||||
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 ⇒ ntrs
|
||||
|
|
|
|||
|
|
@ -53,6 +53,7 @@ import scala.collection.immutable.TreeSet
|
|||
import akka.cluster.MemberStatus
|
||||
import scala.annotation.varargs
|
||||
import akka.util.JavaDurationConverters._
|
||||
import akka.util.ccompat._
|
||||
|
||||
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 = {
|
||||
// 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.
|
||||
DeltaPropagation(selfUniqueAddress, reply = false, deltas.collect {
|
||||
DeltaPropagation(selfUniqueAddress, reply = false, deltas.iterator.collect {
|
||||
case (key, (d, fromSeqNr, toSeqNr)) if d != NoDeltaPlaceholder ⇒
|
||||
getData(key) match {
|
||||
case Some(envelope) ⇒ key → Delta(envelope.copy(data = d), fromSeqNr, toSeqNr)
|
||||
case None ⇒ key → Delta(DataEnvelope(d), fromSeqNr, toSeqNr)
|
||||
}
|
||||
}(collection.breakOut))
|
||||
}.toMap)
|
||||
}
|
||||
}
|
||||
val deltaPropagationTask: Option[Cancellable] =
|
||||
|
|
@ -1461,9 +1462,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
}
|
||||
|
||||
def receiveGetKeyIds(): Unit = {
|
||||
val keys: Set[KeyId] = dataEntries.collect {
|
||||
val keys: Set[KeyId] = dataEntries.iterator.collect {
|
||||
case (key, (DataEnvelope(data, _, _), _)) if data != DeletedData ⇒ key
|
||||
}(collection.breakOut)
|
||||
}.to(immutable.Set)
|
||||
replyTo ! GetKeyIdsResult(keys)
|
||||
}
|
||||
|
||||
|
|
@ -1700,14 +1701,14 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
if (keys.nonEmpty) {
|
||||
if (log.isDebugEnabled)
|
||||
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
|
||||
}
|
||||
val myMissingKeys = otherKeys diff myKeys
|
||||
if (myMissingKeys.nonEmpty) {
|
||||
if (log.isDebugEnabled)
|
||||
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
|
||||
}
|
||||
}
|
||||
|
|
@ -1846,9 +1847,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
|||
|
||||
def initRemovedNodePruning(): Unit = {
|
||||
// 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
|
||||
}(collection.breakOut)
|
||||
}.to(immutable.Set)
|
||||
|
||||
if (removedSet.nonEmpty) {
|
||||
for ((key, (envelope, _)) ← dataEntries; removed ← removedSet) {
|
||||
|
|
|
|||
|
|
@ -12,7 +12,7 @@ import java.util.TreeSet
|
|||
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.collection.breakOut
|
||||
import scala.collection.immutable
|
||||
|
||||
import akka.actor.ExtendedActorSystem
|
||||
import akka.cluster.ddata._
|
||||
|
|
@ -28,6 +28,7 @@ import java.io.NotSerializableException
|
|||
import akka.actor.ActorRef
|
||||
import akka.cluster.ddata.protobuf.msg.ReplicatorMessages.OtherMessage
|
||||
import akka.serialization.Serialization
|
||||
import akka.util.ccompat._
|
||||
|
||||
private object ReplicatedDataSerializer {
|
||||
/*
|
||||
|
|
@ -472,7 +473,7 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
|||
private def orsetDeltaGroupFromBinary(bytes: Array[Byte]): ORSet.DeltaGroup[Any] = {
|
||||
val deltaGroup = rd.ORSetDeltaGroup.parseFrom(bytes)
|
||||
val ops: Vector[ORSet.DeltaOp] =
|
||||
deltaGroup.getEntriesList.asScala.map { entry ⇒
|
||||
deltaGroup.getEntriesList.asScala.iterator.map { entry ⇒
|
||||
if (entry.getOperation == rd.ORSetDeltaOp.Add)
|
||||
ORSet.AddDeltaOp(orsetFromProto(entry.getUnderlying))
|
||||
else if (entry.getOperation == rd.ORSetDeltaOp.Remove)
|
||||
|
|
@ -481,7 +482,7 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
|||
ORSet.FullStateDeltaOp(orsetFromProto(entry.getUnderlying))
|
||||
else
|
||||
throw new NotSerializableException(s"Unknow ORSet delta operation ${entry.getOperation}")
|
||||
}(collection.breakOut)
|
||||
}.to(immutable.Vector)
|
||||
ORSet.DeltaGroup(ops)
|
||||
}
|
||||
|
||||
|
|
@ -538,8 +539,8 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
|||
gcounterFromProto(rd.GCounter.parseFrom(bytes))
|
||||
|
||||
def gcounterFromProto(gcounter: rd.GCounter): GCounter = {
|
||||
new GCounter(state = gcounter.getEntriesList.asScala.map(entry ⇒
|
||||
uniqueAddressFromProto(entry.getNode) → BigInt(entry.getValue.toByteArray))(breakOut))
|
||||
new GCounter(state = gcounter.getEntriesList.asScala.iterator.map(entry ⇒
|
||||
uniqueAddressFromProto(entry.getNode) → BigInt(entry.getValue.toByteArray)).toMap)
|
||||
}
|
||||
|
||||
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] = {
|
||||
val deltaGroup = rd.ORMapDeltaGroup.parseFrom(bytes)
|
||||
val ops: Vector[ORMap.DeltaOp] =
|
||||
deltaGroup.getEntriesList.asScala.map { entry ⇒
|
||||
deltaGroup.getEntriesList.asScala.iterator.map { entry ⇒
|
||||
if (entry.getOperation == rd.ORMapDeltaOp.ORMapPut) {
|
||||
val map = singleMapEntryFromProto(entry.getEntryDataList, (v: dm.OtherMessage) ⇒ otherMessageFromProto(v).asInstanceOf[ReplicatedData])
|
||||
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))
|
||||
} else
|
||||
throw new NotSerializableException(s"Unknown ORMap delta operation ${entry.getOperation}")
|
||||
}(collection.breakOut)
|
||||
}.to(immutable.Vector)
|
||||
ops
|
||||
}
|
||||
|
||||
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 = {
|
||||
ormapDeltaGroupOpsToProto(scala.collection.immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
|
||||
ormapDeltaGroupOpsToProto(immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
|
||||
}
|
||||
|
||||
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 = {
|
||||
ormapDeltaGroupOpsToProto(scala.collection.immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
|
||||
ormapDeltaGroupOpsToProto(immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
|
||||
}
|
||||
|
||||
private def ormapDeltaGroupToProto(deltaGroup: ORMap.DeltaGroup[_, _]): rd.ORMapDeltaGroup = {
|
||||
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) = {
|
||||
if (m.size > 1 && opType != rd.ORMapDeltaOp.ORMapUpdate)
|
||||
throw new IllegalArgumentException("Invalid size of ORMap delta map")
|
||||
|
|
|
|||
|
|
@ -7,7 +7,7 @@ package akka.cluster.ddata.protobuf
|
|||
import scala.concurrent.duration._
|
||||
import java.util.concurrent.TimeUnit
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.collection.breakOut
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration.Duration
|
||||
import akka.actor.ExtendedActorSystem
|
||||
import akka.cluster.Member
|
||||
|
|
@ -32,6 +32,7 @@ import akka.actor.Address
|
|||
import akka.cluster.ddata.VersionVector
|
||||
import akka.annotation.InternalApi
|
||||
import akka.cluster.ddata.PruningState.PruningPerformed
|
||||
import akka.util.ccompat._
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -268,8 +269,8 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
private def statusFromBinary(bytes: Array[Byte]): Status = {
|
||||
val status = dm.Status.parseFrom(bytes)
|
||||
Status(
|
||||
status.getEntriesList.asScala.map(e ⇒
|
||||
e.getKey → AkkaByteString(e.getDigest.toByteArray()))(breakOut),
|
||||
status.getEntriesList.asScala.iterator.map(e ⇒
|
||||
e.getKey → AkkaByteString(e.getDigest.toByteArray())).toMap,
|
||||
status.getChunk, status.getTotChunks)
|
||||
}
|
||||
|
||||
|
|
@ -287,8 +288,8 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
private def gossipFromBinary(bytes: Array[Byte]): Gossip = {
|
||||
val gossip = dm.Gossip.parseFrom(decompress(bytes))
|
||||
Gossip(
|
||||
gossip.getEntriesList.asScala.map(e ⇒
|
||||
e.getKey → dataEnvelopeFromProto(e.getEnvelope))(breakOut),
|
||||
gossip.getEntriesList.asScala.iterator.map(e ⇒
|
||||
e.getKey → dataEnvelopeFromProto(e.getEnvelope)).toMap,
|
||||
sendBack = gossip.getSendBack)
|
||||
}
|
||||
|
||||
|
|
@ -316,11 +317,11 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
DeltaPropagation(
|
||||
uniqueAddressFromProto(deltaPropagation.getFromNode),
|
||||
reply,
|
||||
deltaPropagation.getEntriesList.asScala.map { e ⇒
|
||||
deltaPropagation.getEntriesList.asScala.iterator.map { e ⇒
|
||||
val fromSeqNr = e.getFromSeqNr
|
||||
val toSeqNr = if (e.hasToSeqNr) e.getToSeqNr else fromSeqNr
|
||||
e.getKey → Delta(dataEnvelopeFromProto(e.getEnvelope), fromSeqNr, toSeqNr)
|
||||
}(breakOut))
|
||||
}.toMap)
|
||||
}
|
||||
|
||||
private def getToProto(get: Get[_]): dm.Get = {
|
||||
|
|
@ -482,7 +483,7 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
if (pruningEntries.isEmpty)
|
||||
Map.empty
|
||||
else
|
||||
pruningEntries.asScala.map { pruningEntry ⇒
|
||||
pruningEntries.asScala.iterator.map { pruningEntry ⇒
|
||||
val state =
|
||||
if (pruningEntry.getPerformed) {
|
||||
// for wire compatibility with Akka 2.4.x
|
||||
|
|
@ -491,10 +492,10 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
|||
} else
|
||||
PruningState.PruningInitialized(
|
||||
uniqueAddressFromProto(pruningEntry.getOwnerAddress),
|
||||
pruningEntry.getSeenList.asScala.map(addressFromProto)(breakOut))
|
||||
pruningEntry.getSeenList.asScala.iterator.map(addressFromProto).to(immutable.Set))
|
||||
val removed = uniqueAddressFromProto(pruningEntry.getRemovedAddress)
|
||||
removed → state
|
||||
}(breakOut)
|
||||
}.toMap
|
||||
}
|
||||
|
||||
private def writeToProto(write: Write): dm.Write =
|
||||
|
|
|
|||
|
|
@ -11,7 +11,6 @@ import java.util.zip.GZIPOutputStream
|
|||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable.TreeMap
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.collection.breakOut
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.Address
|
||||
import akka.actor.ExtendedActorSystem
|
||||
|
|
@ -21,6 +20,7 @@ import akka.serialization._
|
|||
import akka.protobuf.ByteString
|
||||
import akka.protobuf.MessageLite
|
||||
import akka.cluster.ddata.VersionVector
|
||||
import akka.util.ccompat._
|
||||
|
||||
/**
|
||||
* Some useful serialization helper methods.
|
||||
|
|
@ -124,8 +124,8 @@ trait SerializationSupport {
|
|||
else if (entries.size == 1)
|
||||
VersionVector(uniqueAddressFromProto(entries.get(0).getNode), entries.get(0).getVersion)
|
||||
else {
|
||||
val versions: TreeMap[UniqueAddress, Long] = versionVector.getEntriesList.asScala.map(entry ⇒
|
||||
uniqueAddressFromProto(entry.getNode) → entry.getVersion)(breakOut)
|
||||
val versions: TreeMap[UniqueAddress, Long] = scala.collection.immutable.TreeMap.from(versionVector.getEntriesList.asScala.iterator.map(entry ⇒
|
||||
uniqueAddressFromProto(entry.getNode) → entry.getVersion))
|
||||
VersionVector(versions)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -136,7 +136,7 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig)
|
|||
expectTerminated(r)
|
||||
|
||||
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
|
||||
r2 ! Get(KeyA, ReadLocal)
|
||||
|
|
@ -184,7 +184,7 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig)
|
|||
expectTerminated(r)
|
||||
|
||||
var r2: ActorRef = null
|
||||
awaitAssert(r2 = newReplicator()) // try until name is free
|
||||
awaitAssert { r2 = newReplicator() } // try until name is free
|
||||
awaitAssert {
|
||||
r2 ! GetKeyIds
|
||||
expectMsgType[GetKeyIdsResult].keyIds should !==(Set.empty[String])
|
||||
|
|
@ -221,7 +221,7 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig)
|
|||
expectTerminated(r)
|
||||
|
||||
var r2: ActorRef = null
|
||||
awaitAssert(r2 = newReplicator()) // try until name is free
|
||||
awaitAssert { r2 = newReplicator() } // try until name is free
|
||||
awaitAssert {
|
||||
r2 ! GetKeyIds
|
||||
expectMsgType[GetKeyIdsResult].keyIds should !==(Set.empty[String])
|
||||
|
|
|
|||
|
|
@ -16,6 +16,7 @@ import akka.actor.ActorSystem
|
|||
import akka.actor.ActorRef
|
||||
import scala.concurrent.Await
|
||||
import akka.cluster.MemberStatus
|
||||
import akka.util.ccompat.imm._
|
||||
|
||||
object DurablePruningSpec extends MultiNodeConfig {
|
||||
val first = role("first")
|
||||
|
|
@ -76,9 +77,9 @@ class DurablePruningSpec extends MultiNodeSpec(DurablePruningSpec) with STMultiN
|
|||
Cluster(sys2).join(node(first).address)
|
||||
awaitAssert({
|
||||
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.map(_.status) should ===(Set(MemberStatus.Up))
|
||||
Cluster(sys2).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
|
||||
}, 10.seconds)
|
||||
enterBarrier("joined")
|
||||
|
||||
|
|
|
|||
|
|
@ -21,8 +21,8 @@ object DeltaPropagationSelectorSpec {
|
|||
override val allNodes: Vector[Address]) extends DeltaPropagationSelector {
|
||||
override val gossipIntervalDivisor = 5
|
||||
override def createDeltaPropagation(deltas: Map[KeyId, (ReplicatedData, Long, Long)]): DeltaPropagation =
|
||||
DeltaPropagation(selfUniqueAddress, false, deltas.mapValues {
|
||||
case (d, fromSeqNr, toSeqNr) ⇒ Delta(DataEnvelope(d), fromSeqNr, toSeqNr)
|
||||
DeltaPropagation(selfUniqueAddress, false, deltas.map {
|
||||
case (key, (d, fromSeqNr, toSeqNr)) ⇒ (key, Delta(DataEnvelope(d), fromSeqNr, toSeqNr))
|
||||
})
|
||||
override def maxDeltaSize: Int = 10
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
val m1 = LWWMap.empty.put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int])
|
||||
val m2 = LWWMap.empty.put(node2, "c", 3, defaultClock[Int])
|
||||
val m1 = LWWMap.empty[String, Int].put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int])
|
||||
val m2 = LWWMap.empty[String, Int].put(node2, "c", 3, defaultClock[Int])
|
||||
|
||||
// merge both ways
|
||||
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 {
|
||||
val m1 = LWWMap.empty.put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int])
|
||||
val m2 = LWWMap.empty.put(node2, "c", 3, defaultClock[Int])
|
||||
val m1 = LWWMap.empty[String, Int].put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int])
|
||||
val m2 = LWWMap.empty[String, Int].put(node2, "c", 3, defaultClock[Int])
|
||||
|
||||
val merged1 = m1 merge m2
|
||||
|
||||
|
|
@ -48,8 +48,8 @@ class LWWMapSpec extends WordSpec with Matchers {
|
|||
}
|
||||
|
||||
"be able to work with deltas" in {
|
||||
val m1 = LWWMap.empty.put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int])
|
||||
val m2 = LWWMap.empty.put(node2, "c", 3, defaultClock[Int])
|
||||
val m1 = LWWMap.empty[String, Int].put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int])
|
||||
val m2 = LWWMap.empty[String, Int].put(node2, "c", 3, defaultClock[Int])
|
||||
|
||||
val expected = Map("a" → 1, "b" → 2, "c" → 3)
|
||||
(m1 merge m2).entries should be(expected)
|
||||
|
|
@ -69,7 +69,7 @@ class LWWMapSpec extends WordSpec with Matchers {
|
|||
}
|
||||
|
||||
"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 entries2: Map[String, Long] = entries1
|
||||
Changed(LWWMapKey[String, Long]("key"))(m1) match {
|
||||
|
|
|
|||
|
|
@ -368,21 +368,7 @@ you can also parallelize it by chunking your futures into sub-sequences and redu
|
|||
## 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.
|
||||
For this `Future` supports `onComplete`, `onSuccess` and `onFailure`, of which the last two are specializations of the first.
|
||||
|
||||
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 }
|
||||
|
||||
For this, `Future` supports `onComplete`:
|
||||
|
||||
Scala
|
||||
: @@snip [FutureDocSpec.scala](/akka-docs/src/test/scala/docs/future/FutureDocSpec.scala) { #onComplete }
|
||||
|
|
|
|||
|
|
@ -18,14 +18,15 @@ import akka.util.Timeout;
|
|||
|
||||
//#imports2
|
||||
import java.time.Duration;
|
||||
import akka.japi.Function;
|
||||
|
||||
import java.util.concurrent.*;
|
||||
|
||||
import scala.util.Try;
|
||||
|
||||
import akka.japi.Function;
|
||||
|
||||
import static akka.dispatch.Futures.future;
|
||||
import static java.util.concurrent.TimeUnit.SECONDS;
|
||||
|
||||
|
||||
//#imports2
|
||||
|
||||
//#imports3
|
||||
|
|
@ -283,7 +284,7 @@ public class FutureDocTest extends AbstractJavaTest {
|
|||
}
|
||||
}, system.dispatcher());
|
||||
|
||||
f.onSuccess(new PrintResult<String>(), system.dispatcher());
|
||||
f.onComplete(new PrintResult<Try<String>>(), system.dispatcher());
|
||||
//#future-eval
|
||||
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
||||
String result = (String) Await.result(f, timeout.duration());
|
||||
|
|
@ -307,7 +308,7 @@ public class FutureDocTest extends AbstractJavaTest {
|
|||
}
|
||||
}, ec);
|
||||
|
||||
f2.onSuccess(new PrintResult<Integer>(), system.dispatcher());
|
||||
f2.onComplete(new PrintResult<Try<Integer>>(), system.dispatcher());
|
||||
//#map
|
||||
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
||||
int result = Await.result(f2, timeout.duration());
|
||||
|
|
@ -335,7 +336,7 @@ public class FutureDocTest extends AbstractJavaTest {
|
|||
}
|
||||
}, ec);
|
||||
|
||||
f2.onSuccess(new PrintResult<Integer>(), system.dispatcher());
|
||||
f2.onComplete(new PrintResult<Try<Integer>>(), system.dispatcher());
|
||||
//#flat-map
|
||||
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
||||
int result = Await.result(f2, timeout.duration());
|
||||
|
|
@ -367,7 +368,7 @@ public class FutureDocTest extends AbstractJavaTest {
|
|||
}
|
||||
}, ec);
|
||||
|
||||
futureSum.onSuccess(new PrintResult<Long>(), system.dispatcher());
|
||||
futureSum.onComplete(new PrintResult<Try<Long>>(), system.dispatcher());
|
||||
//#sequence
|
||||
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
||||
long result = Await.result(futureSum, timeout.duration());
|
||||
|
|
@ -393,7 +394,7 @@ public class FutureDocTest extends AbstractJavaTest {
|
|||
}, ec);
|
||||
|
||||
//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
|
||||
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
||||
Iterable<String> result = Await.result(futureResult, timeout.duration());
|
||||
|
|
@ -420,7 +421,7 @@ public class FutureDocTest extends AbstractJavaTest {
|
|||
}
|
||||
}, ec);
|
||||
|
||||
resultFuture.onSuccess(new PrintResult<String>(), system.dispatcher());
|
||||
resultFuture.onComplete(new PrintResult<Try<String>>(), system.dispatcher());
|
||||
//#fold
|
||||
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
||||
String result = Await.result(resultFuture, timeout.duration());
|
||||
|
|
@ -445,7 +446,7 @@ public class FutureDocTest extends AbstractJavaTest {
|
|||
}
|
||||
}, ec);
|
||||
|
||||
resultFuture.onSuccess(new PrintResult<Object>(), system.dispatcher());
|
||||
resultFuture.onComplete(new PrintResult<Try<Object>>(), system.dispatcher());
|
||||
//#reduce
|
||||
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
||||
Object result = Await.result(resultFuture, timeout.duration());
|
||||
|
|
@ -545,7 +546,7 @@ public class FutureDocTest extends AbstractJavaTest {
|
|||
}
|
||||
}, ec);
|
||||
|
||||
future.onSuccess(new PrintResult<Integer>(), system.dispatcher());
|
||||
future.onComplete(new PrintResult<Try<Integer>>(), system.dispatcher());
|
||||
//#recover
|
||||
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
||||
int result = Await.result(future, timeout.duration());
|
||||
|
|
@ -574,7 +575,7 @@ public class FutureDocTest extends AbstractJavaTest {
|
|||
}
|
||||
}, ec);
|
||||
|
||||
future.onSuccess(new PrintResult<Integer>(), system.dispatcher());
|
||||
future.onComplete(new PrintResult<Try<Integer>>(), system.dispatcher());
|
||||
//#try-recover
|
||||
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
||||
int result = Await.result(future, timeout.duration());
|
||||
|
|
@ -582,42 +583,10 @@ public class FutureDocTest extends AbstractJavaTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void useOnSuccessOnFailureAndOnComplete() throws Exception {
|
||||
public void useOnOnComplete() throws Exception {
|
||||
{
|
||||
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
|
||||
final ExecutionContext ec = system.dispatcher();
|
||||
|
||||
|
|
@ -648,7 +617,7 @@ public class FutureDocTest extends AbstractJavaTest {
|
|||
}
|
||||
}, ec);
|
||||
|
||||
future3.onSuccess(new PrintResult<String>(), system.dispatcher());
|
||||
future3.onComplete(new PrintResult<Try<String>>(), system.dispatcher());
|
||||
//#zip
|
||||
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
||||
String result = Await.result(future3, timeout.duration());
|
||||
|
|
@ -662,7 +631,7 @@ public class FutureDocTest extends AbstractJavaTest {
|
|||
Future<String> future3 = Futures.successful("bar");
|
||||
// Will have "bar" in this case
|
||||
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
|
||||
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
||||
String result = Await.result(future4, timeout.duration());
|
||||
|
|
|
|||
|
|
@ -197,7 +197,7 @@ public class PersistenceQueryDocTest {
|
|||
|
||||
public akka.stream.scaladsl.Source<RichEvent, QueryMetadata> byTagsWithMeta(
|
||||
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();
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -5,6 +5,7 @@
|
|||
package scala.docs.cluster
|
||||
|
||||
import language.postfixOps
|
||||
import scala.util.Success
|
||||
import scala.concurrent.duration._
|
||||
import akka.actor.Actor
|
||||
import akka.actor.ActorRef
|
||||
|
|
@ -55,9 +56,8 @@ object TransformationFrontend {
|
|||
import system.dispatcher
|
||||
system.scheduler.schedule(2.seconds, 2.seconds) {
|
||||
implicit val timeout = Timeout(5 seconds)
|
||||
(frontend ? TransformationJob("hello-" + counter.incrementAndGet())) onSuccess {
|
||||
case result ⇒ println(result)
|
||||
}
|
||||
(frontend ? TransformationJob("hello-" + counter.incrementAndGet()))
|
||||
.foreach { result ⇒ println(result) }
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
Some files were not shown because too many files have changed in this diff Show more
Loading…
Add table
Add a link
Reference in a new issue