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 akka.japi.*;
|
||||||
import org.junit.ClassRule;
|
import org.junit.ClassRule;
|
||||||
import org.scalatest.junit.JUnitSuite;
|
import org.scalatest.junit.JUnitSuite;
|
||||||
|
import scala.Function1;
|
||||||
import scala.concurrent.Await;
|
import scala.concurrent.Await;
|
||||||
import scala.concurrent.Future;
|
import scala.concurrent.Future;
|
||||||
import scala.concurrent.Promise;
|
import scala.concurrent.Promise;
|
||||||
|
|
@ -26,6 +27,7 @@ import java.util.concurrent.TimeUnit;
|
||||||
import static akka.japi.Util.classTag;
|
import static akka.japi.Util.classTag;
|
||||||
|
|
||||||
import akka.testkit.AkkaSpec;
|
import akka.testkit.AkkaSpec;
|
||||||
|
import scala.util.Try;
|
||||||
|
|
||||||
public class JavaFutureTests extends JUnitSuite {
|
public class JavaFutureTests extends JUnitSuite {
|
||||||
|
|
||||||
|
|
@ -59,9 +61,9 @@ public class JavaFutureTests extends JUnitSuite {
|
||||||
final CountDownLatch latch = new CountDownLatch(1);
|
final CountDownLatch latch = new CountDownLatch(1);
|
||||||
Promise<String> cf = Futures.promise();
|
Promise<String> cf = Futures.promise();
|
||||||
Future<String> f = cf.future();
|
Future<String> f = cf.future();
|
||||||
f.onSuccess(new OnSuccess<String>() {
|
f.onComplete(new OnComplete<String>() {
|
||||||
public void onSuccess(String result) {
|
public void onComplete(Throwable t, String r) {
|
||||||
if (result.equals("foo"))
|
if ("foo".equals(r))
|
||||||
latch.countDown();
|
latch.countDown();
|
||||||
}
|
}
|
||||||
}, system.dispatcher());
|
}, system.dispatcher());
|
||||||
|
|
@ -76,8 +78,9 @@ public class JavaFutureTests extends JUnitSuite {
|
||||||
final CountDownLatch latch = new CountDownLatch(1);
|
final CountDownLatch latch = new CountDownLatch(1);
|
||||||
Promise<String> cf = Futures.promise();
|
Promise<String> cf = Futures.promise();
|
||||||
Future<String> f = cf.future();
|
Future<String> f = cf.future();
|
||||||
f.onFailure(new OnFailure() {
|
f.onComplete(new OnComplete<String>() {
|
||||||
public void onFailure(Throwable t) {
|
public void onComplete(Throwable t, String r) {
|
||||||
|
// 'null instanceof ...' is always false
|
||||||
if (t instanceof NullPointerException)
|
if (t instanceof NullPointerException)
|
||||||
latch.countDown();
|
latch.countDown();
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -8,7 +8,7 @@ import language.postfixOps
|
||||||
import java.io.Closeable
|
import java.io.Closeable
|
||||||
import java.util.concurrent._
|
import java.util.concurrent._
|
||||||
import atomic.{ AtomicReference, AtomicInteger }
|
import atomic.{ AtomicReference, AtomicInteger }
|
||||||
import scala.concurrent.{ future, Await, ExecutionContext }
|
import scala.concurrent.{ Future, Await, ExecutionContext }
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import java.util.concurrent.ThreadLocalRandom
|
import java.util.concurrent.ThreadLocalRandom
|
||||||
import scala.util.Try
|
import scala.util.Try
|
||||||
|
|
@ -378,7 +378,7 @@ class LightArrayRevolverSchedulerSpec extends AkkaSpec(SchedulerSpec.testConfRev
|
||||||
import driver._
|
import driver._
|
||||||
import system.dispatcher
|
import system.dispatcher
|
||||||
val counter = new AtomicInteger
|
val counter = new AtomicInteger
|
||||||
val terminated = future {
|
val terminated = Future {
|
||||||
var rounds = 0
|
var rounds = 0
|
||||||
while (Try(sched.scheduleOnce(Duration.Zero)(())(localEC)).isSuccess) {
|
while (Try(sched.scheduleOnce(Duration.Zero)(())(localEC)).isSuccess) {
|
||||||
Thread.sleep(1)
|
Thread.sleep(1)
|
||||||
|
|
@ -511,7 +511,7 @@ class LightArrayRevolverSchedulerSpec extends AkkaSpec(SchedulerSpec.testConfRev
|
||||||
withScheduler() { (sched, driver) ⇒
|
withScheduler() { (sched, driver) ⇒
|
||||||
import system.dispatcher
|
import system.dispatcher
|
||||||
val counter = new AtomicInteger
|
val counter = new AtomicInteger
|
||||||
future { Thread.sleep(5); driver.close(); sched.close() }
|
Future { Thread.sleep(5); driver.close(); sched.close() }
|
||||||
val headroom = 200
|
val headroom = 200
|
||||||
var overrun = headroom
|
var overrun = headroom
|
||||||
val cap = 1000000
|
val cap = 1000000
|
||||||
|
|
|
||||||
|
|
@ -6,7 +6,8 @@ package akka.dispatch
|
||||||
|
|
||||||
import java.util.concurrent.{ ExecutorService, Executor, Executors }
|
import java.util.concurrent.{ ExecutorService, Executor, Executors }
|
||||||
import java.util.concurrent.atomic.AtomicInteger
|
import java.util.concurrent.atomic.AtomicInteger
|
||||||
import scala.concurrent.{ BatchingExecutor => _, Batchable => _, _ }
|
import scala.concurrent.{ ExecutionContext, ExecutionContextExecutor, ExecutionContextExecutorService }
|
||||||
|
import scala.concurrent.{ Await, blocking, Promise, Future }
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import akka.testkit.{ TestLatch, AkkaSpec, DefaultTimeout }
|
import akka.testkit.{ TestLatch, AkkaSpec, DefaultTimeout }
|
||||||
import akka.util.SerializedSuspendableExecutionContext
|
import akka.util.SerializedSuspendableExecutionContext
|
||||||
|
|
|
||||||
|
|
@ -5,7 +5,7 @@
|
||||||
package akka.io
|
package akka.io
|
||||||
|
|
||||||
import akka.testkit.{ TestProbe, AkkaSpec }
|
import akka.testkit.{ TestProbe, AkkaSpec }
|
||||||
import akka.testkit.SocketUtil._
|
import akka.testkit.SocketUtil.temporaryServerAddresses
|
||||||
import Tcp._
|
import Tcp._
|
||||||
|
|
||||||
class CapacityLimitSpec extends AkkaSpec("""
|
class CapacityLimitSpec extends AkkaSpec("""
|
||||||
|
|
|
||||||
|
|
@ -24,7 +24,7 @@ import akka.io.Inet.SocketOption
|
||||||
import akka.actor._
|
import akka.actor._
|
||||||
import akka.testkit.{ AkkaSpec, EventFilter, SocketUtil, TestActorRef, TestProbe }
|
import akka.testkit.{ AkkaSpec, EventFilter, SocketUtil, TestActorRef, TestProbe }
|
||||||
import akka.util.{ ByteString, Helpers }
|
import akka.util.{ ByteString, Helpers }
|
||||||
import akka.testkit.SocketUtil._
|
import akka.testkit.SocketUtil.temporaryServerAddress
|
||||||
import java.util.Random
|
import java.util.Random
|
||||||
import java.net.SocketTimeoutException
|
import java.net.SocketTimeoutException
|
||||||
import java.nio.file.Files
|
import java.nio.file.Files
|
||||||
|
|
|
||||||
|
|
@ -9,11 +9,10 @@ import scala.collection.immutable
|
||||||
import akka.testkit.{ AkkaSpec, TestProbe }
|
import akka.testkit.{ AkkaSpec, TestProbe }
|
||||||
import akka.actor.ActorRef
|
import akka.actor.ActorRef
|
||||||
import akka.io.Inet.SocketOption
|
import akka.io.Inet.SocketOption
|
||||||
import akka.testkit.SocketUtil._
|
import akka.testkit.SocketUtil.temporaryServerAddress
|
||||||
import Tcp._
|
import Tcp._
|
||||||
import akka.actor.ActorSystem
|
import akka.actor.ActorSystem
|
||||||
import akka.dispatch.ExecutionContexts
|
import akka.dispatch.ExecutionContexts
|
||||||
import scala.collection.immutable
|
|
||||||
|
|
||||||
trait TcpIntegrationSpecSupport { _: AkkaSpec ⇒
|
trait TcpIntegrationSpecSupport { _: AkkaSpec ⇒
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -8,7 +8,7 @@ import java.net.InetSocketAddress
|
||||||
import akka.testkit.{ TestProbe, ImplicitSender, AkkaSpec }
|
import akka.testkit.{ TestProbe, ImplicitSender, AkkaSpec }
|
||||||
import akka.util.ByteString
|
import akka.util.ByteString
|
||||||
import akka.actor.ActorRef
|
import akka.actor.ActorRef
|
||||||
import akka.testkit.SocketUtil._
|
import akka.testkit.SocketUtil.temporaryServerAddresses
|
||||||
|
|
||||||
class UdpConnectedIntegrationSpec extends AkkaSpec("""
|
class UdpConnectedIntegrationSpec extends AkkaSpec("""
|
||||||
akka.loglevel = INFO
|
akka.loglevel = INFO
|
||||||
|
|
|
||||||
|
|
@ -11,7 +11,7 @@ import akka.util.ByteString
|
||||||
import akka.actor.ActorRef
|
import akka.actor.ActorRef
|
||||||
import akka.io.Udp._
|
import akka.io.Udp._
|
||||||
import akka.io.Inet._
|
import akka.io.Inet._
|
||||||
import akka.testkit.SocketUtil._
|
import akka.testkit.SocketUtil.temporaryServerAddresses
|
||||||
import java.net.DatagramSocket
|
import java.net.DatagramSocket
|
||||||
|
|
||||||
class UdpIntegrationSpec extends AkkaSpec("""
|
class UdpIntegrationSpec extends AkkaSpec("""
|
||||||
|
|
|
||||||
|
|
@ -53,7 +53,7 @@ class RoundRobinSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
|
||||||
val doneLatch = new TestLatch(connectionCount)
|
val doneLatch = new TestLatch(connectionCount)
|
||||||
|
|
||||||
val counter = new AtomicInteger
|
val counter = new AtomicInteger
|
||||||
var replies = Map.empty[Int, Int].withDefaultValue(0)
|
var replies: Map[Int, Int] = Map.empty.withDefaultValue(0)
|
||||||
|
|
||||||
val actor = system.actorOf(RoundRobinPool(connectionCount).props(routeeProps = Props(new Actor {
|
val actor = system.actorOf(RoundRobinPool(connectionCount).props(routeeProps = Props(new Actor {
|
||||||
lazy val id = counter.getAndIncrement()
|
lazy val id = counter.getAndIncrement()
|
||||||
|
|
@ -65,7 +65,7 @@ class RoundRobinSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
|
||||||
|
|
||||||
for (_ ← 1 to iterationCount; _ ← 1 to connectionCount) {
|
for (_ ← 1 to iterationCount; _ ← 1 to connectionCount) {
|
||||||
val id = Await.result((actor ? "hit").mapTo[Int], timeout.duration)
|
val id = Await.result((actor ? "hit").mapTo[Int], timeout.duration)
|
||||||
replies = replies + (id → (replies(id) + 1))
|
replies += (id → (replies(id) + 1))
|
||||||
}
|
}
|
||||||
|
|
||||||
counter.get should ===(connectionCount)
|
counter.get should ===(connectionCount)
|
||||||
|
|
@ -123,7 +123,7 @@ class RoundRobinSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
|
||||||
val iterationCount = 10
|
val iterationCount = 10
|
||||||
val doneLatch = new TestLatch(connectionCount)
|
val doneLatch = new TestLatch(connectionCount)
|
||||||
|
|
||||||
var replies = Map.empty[String, Int].withDefaultValue(0)
|
var replies: Map[String, Int] = Map.empty.withDefaultValue(0)
|
||||||
|
|
||||||
val paths = (1 to connectionCount) map { n ⇒
|
val paths = (1 to connectionCount) map { n ⇒
|
||||||
val ref = system.actorOf(Props(new Actor {
|
val ref = system.actorOf(Props(new Actor {
|
||||||
|
|
@ -139,7 +139,7 @@ class RoundRobinSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
|
||||||
|
|
||||||
for (_ ← 1 to iterationCount; _ ← 1 to connectionCount) {
|
for (_ ← 1 to iterationCount; _ ← 1 to connectionCount) {
|
||||||
val id = Await.result((actor ? "hit").mapTo[String], timeout.duration)
|
val id = Await.result((actor ? "hit").mapTo[String], timeout.duration)
|
||||||
replies = replies + (id → (replies(id) + 1))
|
replies += (id → (replies(id) + 1))
|
||||||
}
|
}
|
||||||
|
|
||||||
actor ! akka.routing.Broadcast("end")
|
actor ! akka.routing.Broadcast("end")
|
||||||
|
|
@ -154,7 +154,7 @@ class RoundRobinSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
|
||||||
val connectionCount = 10
|
val connectionCount = 10
|
||||||
val iterationCount = 10
|
val iterationCount = 10
|
||||||
|
|
||||||
var replies = Map.empty[String, Int].withDefaultValue(0)
|
var replies: Map[String, Int] = Map.empty.withDefaultValue(0)
|
||||||
|
|
||||||
val actor = system.actorOf(Props(new Actor {
|
val actor = system.actorOf(Props(new Actor {
|
||||||
var n = 0
|
var n = 0
|
||||||
|
|
@ -185,7 +185,7 @@ class RoundRobinSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
|
||||||
|
|
||||||
for (_ ← 1 to iterationCount; _ ← 1 to connectionCount) {
|
for (_ ← 1 to iterationCount; _ ← 1 to connectionCount) {
|
||||||
val id = Await.result((actor ? "hit").mapTo[String], timeout.duration)
|
val id = Await.result((actor ? "hit").mapTo[String], timeout.duration)
|
||||||
replies = replies + (id → (replies(id) + 1))
|
replies += (id → (replies(id) + 1))
|
||||||
}
|
}
|
||||||
|
|
||||||
watch(actor)
|
watch(actor)
|
||||||
|
|
|
||||||
|
|
@ -136,7 +136,7 @@ class BoundedBlockingQueueSpec
|
||||||
}
|
}
|
||||||
|
|
||||||
mustBlockFor(100 milliseconds, f)
|
mustBlockFor(100 milliseconds, f)
|
||||||
events should containInSequence(offer("a"), awaitNotFull, signalNotFull, getSize, awaitNotFull)
|
events.toList should containInSequence(offer("a"), awaitNotFull, signalNotFull, getSize, awaitNotFull)
|
||||||
events shouldNot contain(offer("b"))
|
events shouldNot contain(offer("b"))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -196,7 +196,7 @@ class BoundedBlockingQueueSpec
|
||||||
|
|
||||||
// `f` should still block since the queue is still empty
|
// `f` should still block since the queue is still empty
|
||||||
mustBlockFor(100 milliseconds, f)
|
mustBlockFor(100 milliseconds, f)
|
||||||
events should containInSequence(getSize, awaitNotEmpty, signalNotEmpty, getSize, awaitNotEmpty)
|
events.toList should containInSequence(getSize, awaitNotEmpty, signalNotEmpty, getSize, awaitNotEmpty)
|
||||||
events shouldNot contain(poll)
|
events shouldNot contain(poll)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -297,7 +297,7 @@ class BoundedBlockingQueueSpec
|
||||||
|
|
||||||
// `f` should still block since the queue is still empty
|
// `f` should still block since the queue is still empty
|
||||||
mustBlockFor(100 milliseconds, f)
|
mustBlockFor(100 milliseconds, f)
|
||||||
events should containInSequence(getSize, awaitNotFull, signalNotFull, getSize, awaitNotFull)
|
events.toList should containInSequence(getSize, awaitNotFull, signalNotFull, getSize, awaitNotFull)
|
||||||
events shouldNot contain(offer("World"))
|
events shouldNot contain(offer("World"))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -706,12 +706,12 @@ trait QueueSetupHelper {
|
||||||
|
|
||||||
import akka.util.QueueTestEvents._
|
import akka.util.QueueTestEvents._
|
||||||
|
|
||||||
case class TestContext(queue: BoundedBlockingQueue[String], events: mutable.MutableList[QueueEvent], notEmpty: TestCondition, notFull: TestCondition, lock: ReentrantLock, backingQueue: util.Queue[String])
|
case class TestContext(queue: BoundedBlockingQueue[String], events: mutable.Buffer[QueueEvent], notEmpty: TestCondition, notFull: TestCondition, lock: ReentrantLock, backingQueue: util.Queue[String])
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Backing queue that records all poll and offer calls in `events`
|
* Backing queue that records all poll and offer calls in `events`
|
||||||
*/
|
*/
|
||||||
class TestBackingQueue(events: mutable.MutableList[QueueEvent])
|
class TestBackingQueue(events: mutable.Buffer[QueueEvent])
|
||||||
extends util.LinkedList[String] {
|
extends util.LinkedList[String] {
|
||||||
|
|
||||||
override def poll(): String = {
|
override def poll(): String = {
|
||||||
|
|
@ -733,7 +733,7 @@ trait QueueSetupHelper {
|
||||||
/**
|
/**
|
||||||
* Reentrant lock condition that records when the condition is signaled or `await`ed.
|
* Reentrant lock condition that records when the condition is signaled or `await`ed.
|
||||||
*/
|
*/
|
||||||
class TestCondition(events: mutable.MutableList[QueueEvent], condition: Condition, signalEvent: QueueEvent, awaitEvent: QueueEvent)
|
class TestCondition(events: mutable.Buffer[QueueEvent], condition: Condition, signalEvent: QueueEvent, awaitEvent: QueueEvent)
|
||||||
extends Condition {
|
extends Condition {
|
||||||
|
|
||||||
case class Manual(waitTime: Long = 0, waitingThread: Option[Thread] = None)
|
case class Manual(waitTime: Long = 0, waitingThread: Option[Thread] = None)
|
||||||
|
|
@ -798,7 +798,7 @@ trait QueueSetupHelper {
|
||||||
}
|
}
|
||||||
|
|
||||||
def newBoundedBlockingQueue(maxCapacity: Int): TestContext = {
|
def newBoundedBlockingQueue(maxCapacity: Int): TestContext = {
|
||||||
val events: mutable.MutableList[QueueEvent] = new mutable.MutableList()
|
val events: mutable.Buffer[QueueEvent] = new mutable.ArrayBuffer()
|
||||||
|
|
||||||
val realLock = new ReentrantLock(false)
|
val realLock = new ReentrantLock(false)
|
||||||
val wrappedNotEmpty = new TestCondition(events, realLock.newCondition(), signalNotEmpty, awaitNotEmpty)
|
val wrappedNotEmpty = new TestCondition(events, realLock.newCondition(), signalNotEmpty, awaitNotEmpty)
|
||||||
|
|
|
||||||
|
|
@ -762,11 +762,16 @@ class ByteStringSpec extends WordSpec with Matchers with Checkers {
|
||||||
|
|
||||||
"serialize correctly" when {
|
"serialize correctly" when {
|
||||||
"parsing regular ByteString1C as compat" in {
|
"parsing regular ByteString1C as compat" in {
|
||||||
val oldSerd = "aced000573720021616b6b612e7574696c2e42797465537472696e672442797465537472696e67314336e9eed0afcfe4a40200015b000562797465737400025b427872001b616b6b612e7574696c2e436f6d7061637442797465537472696e67fa2925150f93468f0200007870757200025b42acf317f8060854e002000078700000000a74657374737472696e67"
|
val oldSerd =
|
||||||
|
if (util.Properties.versionNumberString.startsWith("2.11") || util.Properties.versionNumberString.startsWith("2.12"))
|
||||||
|
"aced000573720021616b6b612e7574696c2e42797465537472696e672442797465537472696e67314336e9eed0afcfe4a40200015b000562797465737400025b427872001b616b6b612e7574696c2e436f6d7061637442797465537472696e67fa2925150f93468f0200007870757200025b42acf317f8060854e002000078700000000a74657374737472696e67"
|
||||||
|
else
|
||||||
|
// The data is the same, but the class hierarchy changed in 2.13:
|
||||||
|
"aced000573720021616b6b612e7574696c2e42797465537472696e672442797465537472696e67314336e9eed0afcfe4a40200015b000562797465737400025b427872001b616b6b612e7574696c2e436f6d7061637442797465537472696e676c083a30328adea002000078720014616b6b612e7574696c2e42797465537472696e678efa6cf8286d3c930200007870757200025b42acf317f8060854e002000078700000000a74657374737472696e67"
|
||||||
val bs = ByteString("teststring", "UTF8")
|
val bs = ByteString("teststring", "UTF8")
|
||||||
val str = hexFromSer(bs)
|
val str = hexFromSer(bs)
|
||||||
|
|
||||||
require(oldSerd == str)
|
str should be(oldSerd)
|
||||||
}
|
}
|
||||||
|
|
||||||
"given all types of ByteString" in {
|
"given all types of ByteString" in {
|
||||||
|
|
|
||||||
|
|
@ -11,7 +11,7 @@ import scala.util.Properties
|
||||||
|
|
||||||
class LineNumberSpec extends AkkaSpec {
|
class LineNumberSpec extends AkkaSpec {
|
||||||
|
|
||||||
private val isScala212 = Properties.versionNumberString.startsWith("2.12")
|
private val isScala211 = Properties.versionNumberString.startsWith("2.11")
|
||||||
|
|
||||||
"LineNumbers" when {
|
"LineNumbers" when {
|
||||||
|
|
||||||
|
|
@ -24,11 +24,11 @@ class LineNumberSpec extends AkkaSpec {
|
||||||
|
|
||||||
"work for larger functions" in {
|
"work for larger functions" in {
|
||||||
val result = LineNumbers(twoline)
|
val result = LineNumbers(twoline)
|
||||||
if (isScala212)
|
if (isScala211)
|
||||||
// because how scala 2.12 does the same as Java Lambdas
|
|
||||||
result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 15, 15))
|
|
||||||
else
|
|
||||||
result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 15, 17))
|
result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 15, 17))
|
||||||
|
else
|
||||||
|
// because how scala 2.12+ does the same as Java Lambdas
|
||||||
|
result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 15, 15))
|
||||||
}
|
}
|
||||||
|
|
||||||
"work for partial functions" in {
|
"work for partial functions" in {
|
||||||
|
|
@ -37,11 +37,11 @@ class LineNumberSpec extends AkkaSpec {
|
||||||
|
|
||||||
"work for `def`" in {
|
"work for `def`" in {
|
||||||
val result = LineNumbers(method("foo"))
|
val result = LineNumbers(method("foo"))
|
||||||
if (isScala212)
|
if (isScala211)
|
||||||
|
result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 25, 27))
|
||||||
|
else
|
||||||
// because how scala 2.12 does the same as Java Lambdas
|
// because how scala 2.12 does the same as Java Lambdas
|
||||||
result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 26, 27))
|
result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 26, 27))
|
||||||
else
|
|
||||||
result should ===(SourceFileLines("LineNumberSpecCodeForScala.scala", 25, 27))
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -250,7 +250,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit(
|
||||||
private def nextName(prefix: String = "a"): String = s"$prefix-${nameCounter.next()}"
|
private def nextName(prefix: String = "a"): String = s"$prefix-${nameCounter.next()}"
|
||||||
|
|
||||||
// FIXME eventfilter support in typed testkit
|
// FIXME eventfilter support in typed testkit
|
||||||
import scaladsl.adapter._
|
import akka.actor.typed.scaladsl.adapter._
|
||||||
|
|
||||||
implicit val untypedSystem = system.toUntyped
|
implicit val untypedSystem = system.toUntyped
|
||||||
|
|
||||||
|
|
@ -748,7 +748,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit(
|
||||||
val beh = supervise[String](setup(context ⇒
|
val beh = supervise[String](setup(context ⇒
|
||||||
supervise[String](setup { context ⇒
|
supervise[String](setup { context ⇒
|
||||||
probe.ref ! Started
|
probe.ref ! Started
|
||||||
scaladsl.Behaviors.empty[String]
|
Behaviors.empty[String]
|
||||||
}).onFailure[RuntimeException](strategy)
|
}).onFailure[RuntimeException](strategy)
|
||||||
)).onFailure[Exception](strategy)
|
)).onFailure[Exception](strategy)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -32,18 +32,20 @@ object ReceptionistApiSpec {
|
||||||
// to work
|
// to work
|
||||||
val registered: Future[Receptionist.Registered] =
|
val registered: Future[Receptionist.Registered] =
|
||||||
system.receptionist ? (Receptionist.Register(key, service, _))
|
system.receptionist ? (Receptionist.Register(key, service, _))
|
||||||
registered.onSuccess {
|
registered.foreach {
|
||||||
case key.Registered(ref) ⇒
|
case key.Registered(ref) ⇒
|
||||||
// ref is the right type here
|
// ref is the right type here
|
||||||
ref ! "woho"
|
ref ! "woho"
|
||||||
|
case _ ⇒ ()
|
||||||
}
|
}
|
||||||
|
|
||||||
// one-off ask outside of actor, should be uncommon but not rare
|
// one-off ask outside of actor, should be uncommon but not rare
|
||||||
val found: Future[Receptionist.Listing] =
|
val found: Future[Receptionist.Listing] =
|
||||||
system.receptionist ? (Receptionist.Find(key, _))
|
system.receptionist ? (Receptionist.Find(key, _))
|
||||||
found.onSuccess {
|
found.foreach {
|
||||||
case key.Listing(instances) ⇒
|
case key.Listing(instances) ⇒
|
||||||
instances.foreach(_ ! "woho")
|
instances.foreach(_ ! "woho")
|
||||||
|
case _ ⇒ ()
|
||||||
}
|
}
|
||||||
|
|
||||||
Behaviors.setup[Any] { context ⇒
|
Behaviors.setup[Any] { context ⇒
|
||||||
|
|
|
||||||
|
|
@ -6,7 +6,7 @@ package akka.actor.typed
|
||||||
package scaladsl
|
package scaladsl
|
||||||
|
|
||||||
import akka.actor.ExtendedActorSystem
|
import akka.actor.ExtendedActorSystem
|
||||||
import akka.actor.typed.internal.adapter.{ PropsAdapter => _, _ }
|
import akka.actor.typed.internal.adapter.{ PropsAdapter ⇒ _, _ }
|
||||||
import akka.annotation.InternalApi
|
import akka.annotation.InternalApi
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -72,3 +72,7 @@ ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.pattern.BackoffOption
|
||||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.pattern.HandleBackoff.finalStopMessageReceived")
|
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.pattern.HandleBackoff.finalStopMessageReceived")
|
||||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.pattern.HandleBackoff.finalStopMessageReceived_=")
|
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.pattern.HandleBackoff.finalStopMessageReceived_=")
|
||||||
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.pattern.HandleBackoff.finalStopMessage")
|
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.pattern.HandleBackoff.finalStopMessage")
|
||||||
|
|
||||||
|
# Scala 2.13.0-M5
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.actor.dungeon.ChildrenContainer#ChildrenIterable.this")
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.actor.dungeon.ChildrenContainer#ChildRestartsIterable.this")
|
||||||
|
|
|
||||||
|
|
@ -5,3 +5,7 @@ ProblemFilters.exclude[MissingClassProblem]("akka.io.dns.internal.AsyncDnsResolv
|
||||||
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.io.dns.internal.AsyncDnsCache.put")
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.io.dns.internal.AsyncDnsCache.put")
|
||||||
ProblemFilters.exclude[MissingClassProblem]("akka.io.dns.internal.AsyncDnsResolver$Ipv6Type$")
|
ProblemFilters.exclude[MissingClassProblem]("akka.io.dns.internal.AsyncDnsResolver$Ipv6Type$")
|
||||||
ProblemFilters.exclude[MissingClassProblem]("akka.io.dns.internal.AsyncDnsResolver$Ipv4Type$")
|
ProblemFilters.exclude[MissingClassProblem]("akka.io.dns.internal.AsyncDnsResolver$Ipv4Type$")
|
||||||
|
|
||||||
|
# Changes related to adding Scala 2.13.0-M5 support
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.actor.dungeon.ChildrenContainer#ChildrenIterable.this")
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.actor.dungeon.ChildrenContainer#ChildRestartsIterable.this")
|
||||||
|
|
|
||||||
|
|
@ -1,5 +1,5 @@
|
||||||
/*
|
/*
|
||||||
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
|
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package akka.util
|
package akka.util
|
||||||
|
|
|
||||||
|
|
@ -7,7 +7,7 @@ package akka.compat
|
||||||
import akka.annotation.InternalApi
|
import akka.annotation.InternalApi
|
||||||
import scala.concurrent.{ ExecutionContext, Future ⇒ SFuture }
|
import scala.concurrent.{ ExecutionContext, Future ⇒ SFuture }
|
||||||
import scala.collection.immutable
|
import scala.collection.immutable
|
||||||
import scala.collection.compat._
|
import akka.util.ccompat._
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
|
||||||
|
|
@ -1,5 +1,5 @@
|
||||||
/*
|
/*
|
||||||
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
|
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package akka.util
|
package akka.util
|
||||||
|
|
|
||||||
|
|
@ -12,7 +12,7 @@ import scala.annotation.tailrec
|
||||||
import scala.collection.LinearSeq
|
import scala.collection.LinearSeq
|
||||||
import scala.collection.mutable.ListBuffer
|
import scala.collection.mutable.ListBuffer
|
||||||
import scala.reflect.ClassTag
|
import scala.reflect.ClassTag
|
||||||
import scala.collection.compat._
|
import akka.util.ccompat._
|
||||||
|
|
||||||
object ByteIterator {
|
object ByteIterator {
|
||||||
object ByteArrayIterator {
|
object ByteArrayIterator {
|
||||||
|
|
@ -234,7 +234,7 @@ object ByteIterator {
|
||||||
}
|
}
|
||||||
|
|
||||||
final override def clone: MultiByteArrayIterator = {
|
final override def clone: MultiByteArrayIterator = {
|
||||||
val clonedIterators: List[ByteArrayIterator] = iterators.iterator.map(_.clone).to(scala.collection.immutable.List)
|
val clonedIterators: List[ByteArrayIterator] = iterators.iterator.map(_.clone).to(List)
|
||||||
new MultiByteArrayIterator(clonedIterators)
|
new MultiByteArrayIterator(clonedIterators)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -391,7 +391,6 @@ object ByteIterator {
|
||||||
/**
|
/**
|
||||||
* An iterator over a ByteString.
|
* An iterator over a ByteString.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
abstract class ByteIterator extends BufferedIterator[Byte] {
|
abstract class ByteIterator extends BufferedIterator[Byte] {
|
||||||
def len: Int
|
def len: Int
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -1,5 +1,5 @@
|
||||||
/*
|
/*
|
||||||
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
|
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package akka.util
|
package akka.util
|
||||||
|
|
@ -11,14 +11,13 @@ import java.nio.{ ByteBuffer, ByteOrder }
|
||||||
import java.lang.{ Iterable ⇒ JIterable }
|
import java.lang.{ Iterable ⇒ JIterable }
|
||||||
|
|
||||||
import scala.annotation.{ tailrec, varargs }
|
import scala.annotation.{ tailrec, varargs }
|
||||||
import scala.collection.StrictOptimizedSeqOps
|
|
||||||
import scala.collection.compat._
|
|
||||||
import scala.collection.mutable.{ Builder, WrappedArray }
|
import scala.collection.mutable.{ Builder, WrappedArray }
|
||||||
import scala.collection.{ mutable, immutable }
|
import scala.collection.{ mutable, immutable }
|
||||||
import scala.collection.immutable.{ IndexedSeq, IndexedSeqOps, VectorBuilder }
|
import scala.collection.immutable.{ IndexedSeq, IndexedSeqOps, StrictOptimizedSeqOps, VectorBuilder }
|
||||||
import scala.collection.generic.CanBuildFrom
|
import scala.collection.generic.CanBuildFrom
|
||||||
import scala.reflect.ClassTag
|
import scala.reflect.ClassTag
|
||||||
import java.nio.charset.{ Charset, StandardCharsets }
|
import java.nio.charset.{ Charset, StandardCharsets }
|
||||||
|
import akka.util.ccompat._
|
||||||
|
|
||||||
object ByteString {
|
object ByteString {
|
||||||
|
|
||||||
|
|
@ -669,6 +668,10 @@ sealed abstract class ByteString
|
||||||
override protected def fromSpecific(coll: IterableOnce[Byte]): ByteString = ByteString(coll)
|
override protected def fromSpecific(coll: IterableOnce[Byte]): ByteString = ByteString(coll)
|
||||||
override protected def newSpecificBuilder: mutable.Builder[Byte, ByteString] = ByteString.newBuilder
|
override protected def newSpecificBuilder: mutable.Builder[Byte, ByteString] = ByteString.newBuilder
|
||||||
|
|
||||||
|
// FIXME this is a workaround for
|
||||||
|
// https://github.com/scala/bug/issues/11192#issuecomment-436926231
|
||||||
|
protected[this] override def writeReplace(): AnyRef = this
|
||||||
|
|
||||||
def apply(idx: Int): Byte
|
def apply(idx: Int): Byte
|
||||||
private[akka] def byteStringCompanion: ByteString.Companion
|
private[akka] def byteStringCompanion: ByteString.Companion
|
||||||
// override so that toString will also be `ByteString(...)` for the concrete subclasses
|
// override so that toString will also be `ByteString(...)` for the concrete subclasses
|
||||||
|
|
@ -833,9 +836,7 @@ sealed abstract class ByteString
|
||||||
*/
|
*/
|
||||||
final def mapI(f: Byte ⇒ Int): ByteString = map(f andThen (_.toByte))
|
final def mapI(f: Byte ⇒ Int): ByteString = map(f andThen (_.toByte))
|
||||||
|
|
||||||
def map[A](f: Byte ⇒ Byte): ByteString = {
|
def map[A](f: Byte ⇒ Byte): ByteString = fromSpecific(super.map(f))
|
||||||
???
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
object CompactByteString {
|
object CompactByteString {
|
||||||
|
|
|
||||||
|
|
@ -1,5 +1,5 @@
|
||||||
/*
|
/*
|
||||||
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
|
* Copyright (C) 2018-2019 Lightbend Inc. <https://www.lightbend.com>
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package akka.util.ccompat
|
package akka.util.ccompat
|
||||||
|
|
@ -7,5 +7,4 @@ package akka.util.ccompat
|
||||||
import scala.collection.immutable
|
import scala.collection.immutable
|
||||||
|
|
||||||
package object imm {
|
package object imm {
|
||||||
type MapLike[K, +V, +This <: immutable.Map[K, V]] = immutable.Map[K, V]
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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
|
package akka.util.ccompat
|
||||||
|
|
||||||
import scala.collection.immutable
|
import scala.collection.immutable
|
||||||
|
|
||||||
|
/**
|
||||||
|
* INTERNAL API
|
||||||
|
*/
|
||||||
package object imm {
|
package object imm {
|
||||||
type MapLike[K, +V, +This <: MapLike[K, V, This] with immutable.Map[K, V]] = immutable.MapLike[K, V, This]
|
implicit class SortedSetOps[A](val real: immutable.SortedSet[A]) extends AnyVal {
|
||||||
|
def unsorted: immutable.Set[A] = real
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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
|
package akka.compat
|
||||||
|
|
|
||||||
|
|
@ -22,7 +22,7 @@ import akka.util.{ Helpers, JavaDurationConverters, Timeout }
|
||||||
import akka.dispatch.ExecutionContexts
|
import akka.dispatch.ExecutionContexts
|
||||||
|
|
||||||
import scala.compat.java8.FutureConverters
|
import scala.compat.java8.FutureConverters
|
||||||
import scala.collection.compat._
|
import akka.util.ccompat._
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An ActorSelection is a logical view of a section of an ActorSystem's tree of Actors,
|
* An ActorSelection is a logical view of a section of an ActorSystem's tree of Actors,
|
||||||
|
|
@ -209,7 +209,7 @@ object ActorSelection {
|
||||||
if ((x.indexOf('?') != -1) || (x.indexOf('*') != -1)) SelectChildPattern(x)
|
if ((x.indexOf('?') != -1) || (x.indexOf('*') != -1)) SelectChildPattern(x)
|
||||||
else if (x == "..") SelectParent
|
else if (x == "..") SelectParent
|
||||||
else SelectChildName(x)
|
else SelectChildName(x)
|
||||||
}).to(scala.collection.immutable.IndexedSeq)
|
}).to(immutable.IndexedSeq)
|
||||||
new ActorSelection with ScalaActorSelection {
|
new ActorSelection with ScalaActorSelection {
|
||||||
override val anchor = anchorRef
|
override val anchor = anchorRef
|
||||||
override val path = compiled
|
override val path = compiled
|
||||||
|
|
|
||||||
|
|
@ -17,7 +17,7 @@ import akka.event.Logging.LogEvent
|
||||||
import akka.event.Logging.Error
|
import akka.event.Logging.Error
|
||||||
import akka.event.Logging.Warning
|
import akka.event.Logging.Warning
|
||||||
import scala.util.control.NonFatal
|
import scala.util.control.NonFatal
|
||||||
import scala.collection.compat._
|
import akka.util.ccompat._
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
|
||||||
|
|
@ -8,7 +8,6 @@ import scala.collection.immutable
|
||||||
|
|
||||||
import akka.actor.{ InvalidActorNameException, ChildStats, ChildRestartStats, ChildNameReserved, ActorRef }
|
import akka.actor.{ InvalidActorNameException, ChildStats, ChildRestartStats, ChildNameReserved, ActorRef }
|
||||||
import akka.util.Collections.{ EmptyImmutableSeq, PartialImmutableValuesIterable }
|
import akka.util.Collections.{ EmptyImmutableSeq, PartialImmutableValuesIterable }
|
||||||
import akka.util.ccompat
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
@ -50,13 +49,13 @@ private[akka] object ChildrenContainer {
|
||||||
final case class Creation() extends SuspendReason with WaitingForChildren
|
final case class Creation() extends SuspendReason with WaitingForChildren
|
||||||
case object Termination extends SuspendReason
|
case object Termination extends SuspendReason
|
||||||
|
|
||||||
class ChildRestartsIterable(stats: ccompat.imm.MapLike[_, ChildStats, _]) extends PartialImmutableValuesIterable[ChildStats, ChildRestartStats] {
|
class ChildRestartsIterable(stats: immutable.Map[_, ChildStats]) extends PartialImmutableValuesIterable[ChildStats, ChildRestartStats] {
|
||||||
override final def apply(c: ChildStats) = c.asInstanceOf[ChildRestartStats]
|
override final def apply(c: ChildStats) = c.asInstanceOf[ChildRestartStats]
|
||||||
override final def isDefinedAt(c: ChildStats) = c.isInstanceOf[ChildRestartStats]
|
override final def isDefinedAt(c: ChildStats) = c.isInstanceOf[ChildRestartStats]
|
||||||
override final def valuesIterator = stats.valuesIterator
|
override final def valuesIterator = stats.valuesIterator
|
||||||
}
|
}
|
||||||
|
|
||||||
class ChildrenIterable(stats: ccompat.imm.MapLike[_, ChildStats, _]) extends PartialImmutableValuesIterable[ChildStats, ActorRef] {
|
class ChildrenIterable(stats: immutable.Map[_, ChildStats]) extends PartialImmutableValuesIterable[ChildStats, ActorRef] {
|
||||||
override final def apply(c: ChildStats) = c.asInstanceOf[ChildRestartStats].child
|
override final def apply(c: ChildStats) = c.asInstanceOf[ChildRestartStats].child
|
||||||
override final def isDefinedAt(c: ChildStats) = c.isInstanceOf[ChildRestartStats]
|
override final def isDefinedAt(c: ChildStats) = c.isInstanceOf[ChildRestartStats]
|
||||||
override final def valuesIterator = stats.valuesIterator
|
override final def valuesIterator = stats.valuesIterator
|
||||||
|
|
|
||||||
|
|
@ -16,7 +16,7 @@ import java.util.function.{ Function ⇒ JFunction }
|
||||||
import akka.util.unused
|
import akka.util.unused
|
||||||
|
|
||||||
import scala.collection.immutable
|
import scala.collection.immutable
|
||||||
import scala.collection.compat._
|
import akka.util.ccompat._
|
||||||
|
|
||||||
abstract class Dns {
|
abstract class Dns {
|
||||||
|
|
||||||
|
|
@ -59,10 +59,10 @@ object Dns extends ExtensionId[DnsExt] with ExtensionIdProvider {
|
||||||
def apply(name: String, addresses: Iterable[InetAddress]): Resolved = {
|
def apply(name: String, addresses: Iterable[InetAddress]): Resolved = {
|
||||||
val ipv4: immutable.Seq[Inet4Address] = addresses.iterator.collect({
|
val ipv4: immutable.Seq[Inet4Address] = addresses.iterator.collect({
|
||||||
case a: Inet4Address ⇒ a
|
case a: Inet4Address ⇒ a
|
||||||
}).to(scala.collection.immutable.IndexedSeq)
|
}).to(immutable.IndexedSeq)
|
||||||
val ipv6: immutable.Seq[Inet6Address] = addresses.iterator.collect({
|
val ipv6: immutable.Seq[Inet6Address] = addresses.iterator.collect({
|
||||||
case a: Inet6Address ⇒ a
|
case a: Inet6Address ⇒ a
|
||||||
}).to(scala.collection.immutable.IndexedSeq)
|
}).to(immutable.IndexedSeq)
|
||||||
Resolved(name, ipv4, ipv6)
|
Resolved(name, ipv4, ipv6)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -21,7 +21,6 @@ import java.lang.{ Iterable ⇒ JIterable }
|
||||||
import java.nio.file.Path
|
import java.nio.file.Path
|
||||||
|
|
||||||
import akka.annotation.InternalApi
|
import akka.annotation.InternalApi
|
||||||
import scala.collection.immutable
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* TCP Extension for Akka’s IO layer.
|
* TCP Extension for Akka’s IO layer.
|
||||||
|
|
|
||||||
|
|
@ -22,7 +22,6 @@ import scala.annotation.tailrec
|
||||||
import scala.collection.immutable
|
import scala.collection.immutable
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import scala.util.control.{ NoStackTrace, NonFatal }
|
import scala.util.control.{ NoStackTrace, NonFatal }
|
||||||
import scala.collection.immutable
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Base class for TcpIncomingConnection and TcpOutgoingConnection.
|
* Base class for TcpIncomingConnection and TcpOutgoingConnection.
|
||||||
|
|
|
||||||
|
|
@ -8,7 +8,6 @@ import java.nio.channels.SocketChannel
|
||||||
import scala.collection.immutable
|
import scala.collection.immutable
|
||||||
import akka.actor.ActorRef
|
import akka.actor.ActorRef
|
||||||
import akka.io.Inet.SocketOption
|
import akka.io.Inet.SocketOption
|
||||||
import scala.collection.immutable
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An actor handling the connection state machine for an incoming, already connected
|
* An actor handling the connection state machine for an incoming, already connected
|
||||||
|
|
|
||||||
|
|
@ -12,8 +12,7 @@ import akka.io.Inet.{ SoJavaFactories, SocketOption }
|
||||||
import akka.util.Helpers.Requiring
|
import akka.util.Helpers.Requiring
|
||||||
import akka.util.ByteString
|
import akka.util.ByteString
|
||||||
import akka.actor._
|
import akka.actor._
|
||||||
import scala.collection.immutable
|
import akka.util.ccompat._
|
||||||
import scala.collection.compat._
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* UDP Extension for Akka’s IO layer.
|
* UDP Extension for Akka’s IO layer.
|
||||||
|
|
@ -282,7 +281,7 @@ object UdpMessage {
|
||||||
* message, or the manager will reply with a [[Udp.CommandFailed]] message.
|
* message, or the manager will reply with a [[Udp.CommandFailed]] message.
|
||||||
*/
|
*/
|
||||||
def bind(handler: ActorRef, endpoint: InetSocketAddress, options: JIterable[SocketOption]): Command =
|
def bind(handler: ActorRef, endpoint: InetSocketAddress, options: JIterable[SocketOption]): Command =
|
||||||
Bind(handler, endpoint, options.asScala.to(scala.collection.immutable.IndexedSeq))
|
Bind(handler, endpoint, options.asScala.to(immutable.IndexedSeq))
|
||||||
/**
|
/**
|
||||||
* Bind without specifying options.
|
* Bind without specifying options.
|
||||||
*/
|
*/
|
||||||
|
|
@ -305,7 +304,7 @@ object UdpMessage {
|
||||||
* The “simple sender” will not stop itself, you will have to send it a [[akka.actor.PoisonPill]]
|
* The “simple sender” will not stop itself, you will have to send it a [[akka.actor.PoisonPill]]
|
||||||
* when you want to close the socket.
|
* when you want to close the socket.
|
||||||
*/
|
*/
|
||||||
def simpleSender(options: JIterable[SocketOption]): Command = SimpleSender(options.asScala.to(scala.collection.immutable.IndexedSeq))
|
def simpleSender(options: JIterable[SocketOption]): Command = SimpleSender(options.asScala.to(immutable.IndexedSeq))
|
||||||
/**
|
/**
|
||||||
* Retrieve a simple sender without specifying options.
|
* Retrieve a simple sender without specifying options.
|
||||||
*/
|
*/
|
||||||
|
|
|
||||||
|
|
@ -11,8 +11,7 @@ import akka.io.Inet.SocketOption
|
||||||
import akka.io.Udp.UdpSettings
|
import akka.io.Udp.UdpSettings
|
||||||
import akka.util.ByteString
|
import akka.util.ByteString
|
||||||
import akka.actor._
|
import akka.actor._
|
||||||
import scala.collection.immutable
|
import akka.util.ccompat._
|
||||||
import scala.collection.compat._
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* UDP Extension for Akka’s IO layer.
|
* UDP Extension for Akka’s IO layer.
|
||||||
|
|
|
||||||
|
|
@ -11,7 +11,6 @@ import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
|
||||||
import akka.io.Inet.SocketOption
|
import akka.io.Inet.SocketOption
|
||||||
import akka.io.Udp._
|
import akka.io.Udp._
|
||||||
import akka.actor._
|
import akka.actor._
|
||||||
import scala.collection.immutable
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
|
||||||
|
|
@ -19,7 +19,7 @@ import scala.collection.JavaConverters._
|
||||||
import scala.collection.immutable
|
import scala.collection.immutable
|
||||||
import scala.concurrent.duration.FiniteDuration
|
import scala.concurrent.duration.FiniteDuration
|
||||||
import scala.util.{ Failure, Success, Try }
|
import scala.util.{ Failure, Success, Try }
|
||||||
import scala.collection.compat._
|
import akka.util.ccompat._
|
||||||
|
|
||||||
/** INTERNAL API */
|
/** INTERNAL API */
|
||||||
@InternalApi
|
@InternalApi
|
||||||
|
|
@ -39,7 +39,7 @@ private[dns] final class DnsSettings(system: ExtendedActorSystem, c: Config) {
|
||||||
parseNameserverAddress(other) :: Nil
|
parseNameserverAddress(other) :: Nil
|
||||||
}
|
}
|
||||||
case ConfigValueType.LIST ⇒
|
case ConfigValueType.LIST ⇒
|
||||||
val userAddresses = c.getStringList("nameservers").asScala.iterator.map(parseNameserverAddress).to(scala.collection.immutable.IndexedSeq)
|
val userAddresses = c.getStringList("nameservers").asScala.iterator.map(parseNameserverAddress).to(immutable.IndexedSeq)
|
||||||
require(userAddresses.nonEmpty, "nameservers can not be empty")
|
require(userAddresses.nonEmpty, "nameservers can not be empty")
|
||||||
userAddresses.toList
|
userAddresses.toList
|
||||||
case _ ⇒ throw new IllegalArgumentException("Invalid type for nameservers. Must be a string or string list")
|
case _ ⇒ throw new IllegalArgumentException("Invalid type for nameservers. Must be a string or string list")
|
||||||
|
|
|
||||||
|
|
@ -92,10 +92,7 @@ private[io] final class AsyncDnsResolver(
|
||||||
|
|
||||||
private def sendQuestion(resolver: ActorRef, message: DnsQuestion): Future[Answer] = {
|
private def sendQuestion(resolver: ActorRef, message: DnsQuestion): Future[Answer] = {
|
||||||
val result = (resolver ? message).mapTo[Answer]
|
val result = (resolver ? message).mapTo[Answer]
|
||||||
result.onComplete {
|
result.failed.foreach { _ ⇒ resolver ! DropRequest(message.id) }
|
||||||
case scala.util.Failure(NonFatal(_)) ⇒ resolver ! DropRequest(message.id)
|
|
||||||
case _ ⇒ ()
|
|
||||||
}
|
|
||||||
result
|
result
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -21,7 +21,7 @@
|
||||||
package akka.routing
|
package akka.routing
|
||||||
|
|
||||||
import java.lang.Integer.{ rotateLeft ⇒ rotl }
|
import java.lang.Integer.{ rotateLeft ⇒ rotl }
|
||||||
import scala.collection.compat._
|
import akka.util.ccompat._
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An object designed to generate well-distributed non-cryptographic
|
* An object designed to generate well-distributed non-cryptographic
|
||||||
|
|
|
||||||
|
|
@ -20,7 +20,7 @@ import akka.dispatch.MessageDispatcher
|
||||||
|
|
||||||
import scala.collection.immutable
|
import scala.collection.immutable
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import scala.collection.compat._
|
import akka.util.ccompat._
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
@ -110,7 +110,7 @@ private[akka] class RoutedActorCell(
|
||||||
case group: Group ⇒
|
case group: Group ⇒
|
||||||
val paths = group.paths(system)
|
val paths = group.paths(system)
|
||||||
if (paths.nonEmpty)
|
if (paths.nonEmpty)
|
||||||
addRoutees(paths.iterator.map(p ⇒ group.routeeFor(p, this)).to(scala.collection.immutable.IndexedSeq))
|
addRoutees(paths.iterator.map(p ⇒ group.routeeFor(p, this)).to(immutable.IndexedSeq))
|
||||||
case _ ⇒
|
case _ ⇒
|
||||||
}
|
}
|
||||||
preSuperStart()
|
preSuperStart()
|
||||||
|
|
|
||||||
|
|
@ -22,7 +22,7 @@ import java.util.concurrent.atomic.AtomicReference
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
import java.util.NoSuchElementException
|
import java.util.NoSuchElementException
|
||||||
import akka.annotation.InternalApi
|
import akka.annotation.InternalApi
|
||||||
import scala.collection.compat._
|
import akka.util.ccompat._
|
||||||
|
|
||||||
object Serialization {
|
object Serialization {
|
||||||
|
|
||||||
|
|
@ -376,6 +376,8 @@ class Serialization(val system: ExtendedActorSystem) extends Extension {
|
||||||
system.dynamicAccess.createInstanceFor[Serializer](fqn, List(classOf[ExtendedActorSystem] → system)) recoverWith {
|
system.dynamicAccess.createInstanceFor[Serializer](fqn, List(classOf[ExtendedActorSystem] → system)) recoverWith {
|
||||||
case _: NoSuchMethodException ⇒
|
case _: NoSuchMethodException ⇒
|
||||||
system.dynamicAccess.createInstanceFor[Serializer](fqn, Nil)
|
system.dynamicAccess.createInstanceFor[Serializer](fqn, Nil)
|
||||||
|
// FIXME only needed on 2.13.0-M5 due to https://github.com/scala/bug/issues/11242
|
||||||
|
case t ⇒ Failure(t)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -5,7 +5,7 @@
|
||||||
package akka.util
|
package akka.util
|
||||||
|
|
||||||
import scala.collection.immutable
|
import scala.collection.immutable
|
||||||
import scala.collection.compat._
|
import akka.util.ccompat._
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Typeclass which describes a classification hierarchy. Observe the contract between `isEqual` and `isSubclass`!
|
* Typeclass which describes a classification hierarchy. Observe the contract between `isEqual` and `isSubclass`!
|
||||||
|
|
|
||||||
|
|
@ -95,7 +95,7 @@ class CamelSettings private[camel] (config: Config, dynamicAccess: DynamicAccess
|
||||||
val section = config.getConfig("akka.camel.conversions")
|
val section = config.getConfig("akka.camel.conversions")
|
||||||
section.entrySet.asScala.map(e ⇒ (e.getKey, section.getString(e.getKey)))
|
section.entrySet.asScala.map(e ⇒ (e.getKey, section.getString(e.getKey)))
|
||||||
}
|
}
|
||||||
val conversions = (Map[String, Class[_ <: AnyRef]]() /: specifiedConversions) {
|
val conversions = specifiedConversions.foldLeft(Map[String, Class[_ <: AnyRef]]()) {
|
||||||
case (m, (key, fqcn)) ⇒
|
case (m, (key, fqcn)) ⇒
|
||||||
m.updated(key, dynamicAccess.getClassFor[AnyRef](fqcn).recover {
|
m.updated(key, dynamicAccess.getClassFor[AnyRef](fqcn).recover {
|
||||||
case e ⇒ throw new ConfigurationException("Could not find/load Camel Converter class [" + fqcn + "]", e)
|
case e ⇒ throw new ConfigurationException("Could not find/load Camel Converter class [" + fqcn + "]", e)
|
||||||
|
|
|
||||||
|
|
@ -11,16 +11,16 @@ import akka.AkkaException
|
||||||
import scala.reflect.ClassTag
|
import scala.reflect.ClassTag
|
||||||
import scala.runtime.ScalaRunTime
|
import scala.runtime.ScalaRunTime
|
||||||
import scala.util.Try
|
import scala.util.Try
|
||||||
import scala.collection.JavaConversions._
|
|
||||||
import akka.dispatch.Mapper
|
import akka.dispatch.Mapper
|
||||||
|
import scala.collection.JavaConverters._
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An immutable representation of a Camel message.
|
* An immutable representation of a Camel message.
|
||||||
*/
|
*/
|
||||||
@deprecated("Akka Camel is deprecated in favour of 'Alpakka', the Akka Streams based collection of integrations to various endpoints (including Camel).", since = "2.5.0")
|
@deprecated("Akka Camel is deprecated in favour of 'Alpakka', the Akka Streams based collection of integrations to various endpoints (including Camel).", since = "2.5.0")
|
||||||
class CamelMessage(val body: Any, val headers: Map[String, Any], val attachments: Map[String, DataHandler]) extends Serializable with Product {
|
class CamelMessage(val body: Any, val headers: Map[String, Any], val attachments: Map[String, DataHandler]) extends Serializable with Product {
|
||||||
def this(body: Any, headers: JMap[String, Any]) = this(body, headers.toMap, Map.empty[String, DataHandler]) //Java
|
def this(body: Any, headers: JMap[String, Any]) = this(body, headers.asScala.toMap, Map.empty[String, DataHandler]) //Java
|
||||||
def this(body: Any, headers: JMap[String, Any], attachments: JMap[String, DataHandler]) = this(body, headers.toMap, attachments.toMap) //Java
|
def this(body: Any, headers: JMap[String, Any], attachments: JMap[String, DataHandler]) = this(body, headers.asScala.toMap, attachments.asScala.toMap) //Java
|
||||||
def this(body: Any, headers: Map[String, Any]) = this(body, headers.toMap, Map.empty[String, DataHandler])
|
def this(body: Any, headers: Map[String, Any]) = this(body, headers.toMap, Map.empty[String, DataHandler])
|
||||||
|
|
||||||
def copy(body: Any = this.body, headers: Map[String, Any] = this.headers): CamelMessage = CamelMessage(body, headers, this.attachments)
|
def copy(body: Any = this.body, headers: Map[String, Any] = this.headers): CamelMessage = CamelMessage(body, headers, this.attachments)
|
||||||
|
|
@ -37,19 +37,19 @@ class CamelMessage(val body: Any, val headers: Map[String, Any], val attachments
|
||||||
* The returned headers map is backed up by an immutable headers map. Any attempt to modify
|
* The returned headers map is backed up by an immutable headers map. Any attempt to modify
|
||||||
* the returned map will throw an exception.
|
* the returned map will throw an exception.
|
||||||
*/
|
*/
|
||||||
def getHeaders(names: JSet[String]): JMap[String, Any] = headers(names.toSet)
|
def getHeaders(names: JSet[String]): JMap[String, Any] = headers(names.asScala.toSet).asJava
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API: Returns all headers from this message. The returned headers map is backed up by this
|
* Java API: Returns all headers from this message. The returned headers map is backed up by this
|
||||||
* message's immutable headers map. Any attempt to modify the returned map will throw an
|
* message's immutable headers map. Any attempt to modify the returned map will throw an
|
||||||
* exception.
|
* exception.
|
||||||
*/
|
*/
|
||||||
def getHeaders: JMap[String, Any] = headers
|
def getHeaders: JMap[String, Any] = headers.asJava
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API: Creates a new CamelMessage with given <code>headers</code>. A copy of the headers map is made.
|
* Java API: Creates a new CamelMessage with given <code>headers</code>. A copy of the headers map is made.
|
||||||
*/
|
*/
|
||||||
def withHeaders[A](headers: JMap[String, A]): CamelMessage = copy(this.body, headers.toMap)
|
def withHeaders[A](headers: JMap[String, A]): CamelMessage = copy(this.body, headers.asScala.toMap)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the header by given <code>name</code> parameter in a [[scala.util.Try]]. The header is converted to type <code>T</code>, which is returned
|
* Returns the header by given <code>name</code> parameter in a [[scala.util.Try]]. The header is converted to type <code>T</code>, which is returned
|
||||||
|
|
@ -145,19 +145,19 @@ class CamelMessage(val body: Any, val headers: Map[String, Any], val attachments
|
||||||
* The returned headers map is backed up by an immutable headers map. Any attempt to modify
|
* The returned headers map is backed up by an immutable headers map. Any attempt to modify
|
||||||
* the returned map will throw an exception.
|
* the returned map will throw an exception.
|
||||||
*/
|
*/
|
||||||
def getAttachments(names: JSet[String]): JMap[String, DataHandler] = attachments(names.toSet)
|
def getAttachments(names: JSet[String]): JMap[String, DataHandler] = attachments(names.asScala.toSet).asJava
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API: Returns all attachments from this message. The returned attachments map is backed up by this
|
* Java API: Returns all attachments from this message. The returned attachments map is backed up by this
|
||||||
* message's immutable headers map. Any attempt to modify the returned map will throw an
|
* message's immutable headers map. Any attempt to modify the returned map will throw an
|
||||||
* exception.
|
* exception.
|
||||||
*/
|
*/
|
||||||
def getAttachments: JMap[String, DataHandler] = attachments
|
def getAttachments: JMap[String, DataHandler] = attachments.asJava
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API: Creates a new CamelMessage with given <code>attachments</code>. A copy of the attachments map is made.
|
* Java API: Creates a new CamelMessage with given <code>attachments</code>. A copy of the attachments map is made.
|
||||||
*/
|
*/
|
||||||
def withAttachments(attachments: JMap[String, DataHandler]): CamelMessage = CamelMessage(this.body, this.headers, attachments.toMap)
|
def withAttachments(attachments: JMap[String, DataHandler]): CamelMessage = CamelMessage(this.body, this.headers, attachments.asScala.toMap)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* SCALA API: Creates a new CamelMessage with given <code>attachments</code>.
|
* SCALA API: Creates a new CamelMessage with given <code>attachments</code>.
|
||||||
|
|
@ -250,7 +250,7 @@ object CamelMessage extends ((Any, Map[String, Any]) ⇒ CamelMessage) {
|
||||||
* in the Camel message.
|
* in the Camel message.
|
||||||
*/
|
*/
|
||||||
private[camel] def from(camelMessage: JCamelMessage, headers: Map[String, Any]): CamelMessage =
|
private[camel] def from(camelMessage: JCamelMessage, headers: Map[String, Any]): CamelMessage =
|
||||||
CamelMessage(camelMessage.getBody, headers ++ camelMessage.getHeaders, camelMessage.getAttachments.toMap)
|
CamelMessage(camelMessage.getBody, headers ++ camelMessage.getHeaders.asScala, camelMessage.getAttachments.asScala.toMap)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a new CamelMessageWithAttachments object from the Camel message.
|
* Creates a new CamelMessageWithAttachments object from the Camel message.
|
||||||
|
|
@ -261,7 +261,7 @@ object CamelMessage extends ((Any, Map[String, Any]) ⇒ CamelMessage) {
|
||||||
* in the Camel message.
|
* in the Camel message.
|
||||||
*/
|
*/
|
||||||
private[camel] def from(camelMessage: JCamelMessage, headers: Map[String, Any], attachments: Map[String, DataHandler]): CamelMessage =
|
private[camel] def from(camelMessage: JCamelMessage, headers: Map[String, Any], attachments: Map[String, DataHandler]): CamelMessage =
|
||||||
CamelMessage(camelMessage.getBody, headers ++ camelMessage.getHeaders, attachments ++ camelMessage.getAttachments)
|
CamelMessage(camelMessage.getBody, headers ++ camelMessage.getHeaders.asScala, attachments ++ camelMessage.getAttachments.asScala)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
|
||||||
|
|
@ -368,7 +368,8 @@ abstract class MixMetricsSelectorBase(selectors: immutable.IndexedSeq[CapacityMe
|
||||||
override def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] = {
|
override def capacity(nodeMetrics: Set[NodeMetrics]): Map[Address, Double] = {
|
||||||
val combined: immutable.IndexedSeq[(Address, Double)] = selectors.flatMap(_.capacity(nodeMetrics).toSeq)
|
val combined: immutable.IndexedSeq[(Address, Double)] = selectors.flatMap(_.capacity(nodeMetrics).toSeq)
|
||||||
// aggregated average of the capacities by address
|
// aggregated average of the capacities by address
|
||||||
combined.foldLeft(Map.empty[Address, (Double, Int)].withDefaultValue((0.0, 0))) {
|
val init: Map[Address, (Double, Int)] = Map.empty.withDefaultValue((0.0, 0))
|
||||||
|
combined.foldLeft(init) {
|
||||||
case (acc, (address, capacity)) ⇒
|
case (acc, (address, capacity)) ⇒
|
||||||
val (sum, count) = acc(address)
|
val (sum, count) = acc(address)
|
||||||
acc + (address → ((sum + capacity, count + 1)))
|
acc + (address → ((sum + capacity, count + 1)))
|
||||||
|
|
|
||||||
|
|
@ -282,8 +282,8 @@ final case class NodeMetrics(address: Address, timestamp: Long, metrics: Set[Met
|
||||||
require(address == that.address, s"merge only allowed for same address, [$address] != [$that.address]")
|
require(address == that.address, s"merge only allowed for same address, [$address] != [$that.address]")
|
||||||
if (timestamp >= that.timestamp) this // that is older
|
if (timestamp >= that.timestamp) this // that is older
|
||||||
else {
|
else {
|
||||||
// equality is based on the name of the Metric and Set doesn't replace existing element
|
// equality is based on the name of the Metric
|
||||||
copy(metrics = that.metrics union metrics, timestamp = that.timestamp)
|
copy(metrics = that.metrics union (metrics diff that.metrics), timestamp = that.timestamp)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -303,8 +303,8 @@ final case class NodeMetrics(address: Address, timestamp: Long, metrics: Set[Met
|
||||||
current :+ latest
|
current :+ latest
|
||||||
}
|
}
|
||||||
// Append metrics missing from either latest or current.
|
// Append metrics missing from either latest or current.
|
||||||
// Equality is based on the [[Metric.name]] and [[Set]] doesn't replace existing elements.
|
// Equality is based on the [[Metric.name]]
|
||||||
val merged = updated union latestNode.metrics union currentNode.metrics
|
val merged = updated union (latestNode.metrics diff updated) union (currentNode.metrics diff updated diff latestNode.metrics)
|
||||||
copy(metrics = merged, timestamp = latestNode.timestamp)
|
copy(metrics = merged, timestamp = latestNode.timestamp)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -14,8 +14,10 @@ import akka.cluster.metrics._
|
||||||
import akka.serialization.{ BaseSerializer, SerializationExtension, Serializers, SerializerWithStringManifest }
|
import akka.serialization.{ BaseSerializer, SerializationExtension, Serializers, SerializerWithStringManifest }
|
||||||
import akka.util.ClassLoaderObjectInputStream
|
import akka.util.ClassLoaderObjectInputStream
|
||||||
import akka.protobuf.{ ByteString, MessageLite }
|
import akka.protobuf.{ ByteString, MessageLite }
|
||||||
|
import akka.util.ccompat._
|
||||||
|
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
|
import scala.collection.immutable
|
||||||
import scala.collection.JavaConverters.{ asJavaIterableConverter, asScalaBufferConverter, setAsJavaSetConverter }
|
import scala.collection.JavaConverters.{ asJavaIterableConverter, asScalaBufferConverter, setAsJavaSetConverter }
|
||||||
import java.io.NotSerializableException
|
import java.io.NotSerializableException
|
||||||
|
|
||||||
|
|
@ -170,9 +172,8 @@ class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithS
|
||||||
}
|
}
|
||||||
|
|
||||||
private def metricsGossipEnvelopeToProto(envelope: MetricsGossipEnvelope): cm.MetricsGossipEnvelope = {
|
private def metricsGossipEnvelopeToProto(envelope: MetricsGossipEnvelope): cm.MetricsGossipEnvelope = {
|
||||||
import scala.collection.breakOut
|
|
||||||
val allNodeMetrics = envelope.gossip.nodes
|
val allNodeMetrics = envelope.gossip.nodes
|
||||||
val allAddresses: Vector[Address] = allNodeMetrics.map(_.address)(breakOut)
|
val allAddresses: Vector[Address] = allNodeMetrics.iterator.map(_.address).to(immutable.Vector)
|
||||||
val addressMapping = allAddresses.zipWithIndex.toMap
|
val addressMapping = allAddresses.zipWithIndex.toMap
|
||||||
val allMetricNames: Vector[String] = allNodeMetrics.foldLeft(Set.empty[String])((s, n) ⇒ s ++ n.metrics.iterator.map(_.name)).toVector
|
val allMetricNames: Vector[String] = allNodeMetrics.foldLeft(Set.empty[String])((s, n) ⇒ s ++ n.metrics.iterator.map(_.name)).toVector
|
||||||
val metricNamesMapping = allMetricNames.zipWithIndex.toMap
|
val metricNamesMapping = allMetricNames.zipWithIndex.toMap
|
||||||
|
|
@ -221,9 +222,8 @@ class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithS
|
||||||
metricsGossipEnvelopeFromProto(cm.MetricsGossipEnvelope.parseFrom(decompress(bytes)))
|
metricsGossipEnvelopeFromProto(cm.MetricsGossipEnvelope.parseFrom(decompress(bytes)))
|
||||||
|
|
||||||
private def metricsGossipEnvelopeFromProto(envelope: cm.MetricsGossipEnvelope): MetricsGossipEnvelope = {
|
private def metricsGossipEnvelopeFromProto(envelope: cm.MetricsGossipEnvelope): MetricsGossipEnvelope = {
|
||||||
import scala.collection.breakOut
|
|
||||||
val mgossip = envelope.getGossip
|
val mgossip = envelope.getGossip
|
||||||
val addressMapping: Vector[Address] = mgossip.getAllAddressesList.asScala.map(addressFromProto)(breakOut)
|
val addressMapping: Vector[Address] = mgossip.getAllAddressesList.asScala.iterator.map(addressFromProto).to(Vector)
|
||||||
val metricNameMapping: Vector[String] = mgossip.getAllMetricNamesList.asScala.toVector
|
val metricNameMapping: Vector[String] = mgossip.getAllMetricNamesList.asScala.toVector
|
||||||
|
|
||||||
def ewmaFromProto(ewma: cm.NodeMetrics.EWMA): Option[EWMA] =
|
def ewmaFromProto(ewma: cm.NodeMetrics.EWMA): Option[EWMA] =
|
||||||
|
|
@ -252,9 +252,9 @@ class MessageSerializer(val system: ExtendedActorSystem) extends SerializerWithS
|
||||||
|
|
||||||
def nodeMetricsFromProto(nodeMetrics: cm.NodeMetrics): NodeMetrics =
|
def nodeMetricsFromProto(nodeMetrics: cm.NodeMetrics): NodeMetrics =
|
||||||
NodeMetrics(addressMapping(nodeMetrics.getAddressIndex), nodeMetrics.getTimestamp,
|
NodeMetrics(addressMapping(nodeMetrics.getAddressIndex), nodeMetrics.getTimestamp,
|
||||||
nodeMetrics.getMetricsList.asScala.map(metricFromProto)(breakOut))
|
nodeMetrics.getMetricsList.asScala.iterator.map(metricFromProto).to(immutable.Set))
|
||||||
|
|
||||||
val nodeMetrics: Set[NodeMetrics] = mgossip.getNodeMetricsList.asScala.map(nodeMetricsFromProto)(breakOut)
|
val nodeMetrics: Set[NodeMetrics] = mgossip.getNodeMetricsList.asScala.iterator.map(nodeMetricsFromProto).to(immutable.Set)
|
||||||
|
|
||||||
MetricsGossipEnvelope(addressFromProto(envelope.getFrom), MetricsGossip(nodeMetrics), envelope.getReply)
|
MetricsGossipEnvelope(addressFromProto(envelope.getFrom), MetricsGossip(nodeMetrics), envelope.getReply)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -31,6 +31,7 @@ import akka.serialization.SerializerWithStringManifest
|
||||||
import akka.util.Timeout
|
import akka.util.Timeout
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
import org.scalatest.WordSpecLike
|
import org.scalatest.WordSpecLike
|
||||||
|
import akka.util.ccompat.imm._
|
||||||
|
|
||||||
object ClusterShardingSpec {
|
object ClusterShardingSpec {
|
||||||
val config = ConfigFactory.parseString(
|
val config = ConfigFactory.parseString(
|
||||||
|
|
@ -236,11 +237,11 @@ class ClusterShardingSpec extends ScalaTestWithActorTestKit(ClusterShardingSpec.
|
||||||
Cluster(system2).manager ! Join(Cluster(system).selfMember.address)
|
Cluster(system2).manager ! Join(Cluster(system).selfMember.address)
|
||||||
|
|
||||||
eventually {
|
eventually {
|
||||||
Cluster(system).state.members.map(_.status) should ===(Set[MemberStatus](MemberStatus.Up))
|
Cluster(system).state.members.unsorted.map(_.status) should ===(Set[MemberStatus](MemberStatus.Up))
|
||||||
Cluster(system).state.members.size should ===(2)
|
Cluster(system).state.members.size should ===(2)
|
||||||
}
|
}
|
||||||
eventually {
|
eventually {
|
||||||
Cluster(system2).state.members.map(_.status) should ===(Set[MemberStatus](MemberStatus.Up))
|
Cluster(system2).state.members.unsorted.map(_.status) should ===(Set[MemberStatus](MemberStatus.Up))
|
||||||
Cluster(system2).state.members.size should ===(2)
|
Cluster(system2).state.members.size should ===(2)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -10,7 +10,7 @@ import java.util.zip.GZIPOutputStream
|
||||||
|
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
import scala.collection.JavaConverters._
|
import scala.collection.JavaConverters._
|
||||||
import scala.collection.breakOut
|
import scala.collection.immutable
|
||||||
import akka.actor.ActorRef
|
import akka.actor.ActorRef
|
||||||
import akka.actor.ExtendedActorSystem
|
import akka.actor.ExtendedActorSystem
|
||||||
import akka.cluster.sharding.Shard
|
import akka.cluster.sharding.Shard
|
||||||
|
|
@ -20,6 +20,7 @@ import akka.serialization.BaseSerializer
|
||||||
import akka.serialization.Serialization
|
import akka.serialization.Serialization
|
||||||
import akka.serialization.SerializerWithStringManifest
|
import akka.serialization.SerializerWithStringManifest
|
||||||
import akka.protobuf.MessageLite
|
import akka.protobuf.MessageLite
|
||||||
|
import akka.util.ccompat._
|
||||||
import java.io.NotSerializableException
|
import java.io.NotSerializableException
|
||||||
|
|
||||||
import akka.cluster.sharding.ShardRegion._
|
import akka.cluster.sharding.ShardRegion._
|
||||||
|
|
@ -214,16 +215,16 @@ private[akka] class ClusterShardingMessageSerializer(val system: ExtendedActorSy
|
||||||
|
|
||||||
private def coordinatorStateFromProto(state: sm.CoordinatorState): State = {
|
private def coordinatorStateFromProto(state: sm.CoordinatorState): State = {
|
||||||
val shards: Map[String, ActorRef] =
|
val shards: Map[String, ActorRef] =
|
||||||
state.getShardsList.asScala.toVector.map { entry ⇒
|
state.getShardsList.asScala.toVector.iterator.map { entry ⇒
|
||||||
entry.getShardId → resolveActorRef(entry.getRegionRef)
|
entry.getShardId → resolveActorRef(entry.getRegionRef)
|
||||||
}(breakOut)
|
}.toMap
|
||||||
|
|
||||||
val regionsZero: Map[ActorRef, Vector[String]] =
|
val regionsZero: Map[ActorRef, Vector[String]] =
|
||||||
state.getRegionsList.asScala.toVector.map(resolveActorRef(_) → Vector.empty[String])(breakOut)
|
state.getRegionsList.asScala.toVector.iterator.map(resolveActorRef(_) → Vector.empty[String]).toMap
|
||||||
val regions: Map[ActorRef, Vector[String]] =
|
val regions: Map[ActorRef, Vector[String]] =
|
||||||
shards.foldLeft(regionsZero) { case (acc, (shardId, regionRef)) ⇒ acc.updated(regionRef, acc(regionRef) :+ shardId) }
|
shards.foldLeft(regionsZero) { case (acc, (shardId, regionRef)) ⇒ acc.updated(regionRef, acc(regionRef) :+ shardId) }
|
||||||
|
|
||||||
val proxies: Set[ActorRef] = state.getRegionProxiesList.asScala.map { resolveActorRef }(breakOut)
|
val proxies: Set[ActorRef] = state.getRegionProxiesList.asScala.iterator.map { resolveActorRef }.to(immutable.Set)
|
||||||
val unallocatedShards: Set[String] = state.getUnallocatedShardsList.asScala.toSet
|
val unallocatedShards: Set[String] = state.getUnallocatedShardsList.asScala.toSet
|
||||||
|
|
||||||
State(shards, regions, proxies, unallocatedShards)
|
State(shards, regions, proxies, unallocatedShards)
|
||||||
|
|
@ -299,7 +300,7 @@ private[akka] class ClusterShardingMessageSerializer(val system: ExtendedActorSy
|
||||||
|
|
||||||
private def shardRegionStatsFromBinary(bytes: Array[Byte]): ShardRegionStats = {
|
private def shardRegionStatsFromBinary(bytes: Array[Byte]): ShardRegionStats = {
|
||||||
val parsed = sm.ShardRegionStats.parseFrom(bytes)
|
val parsed = sm.ShardRegionStats.parseFrom(bytes)
|
||||||
val stats: Map[String, Int] = parsed.getStatsList.asScala.map(e ⇒ e.getKey -> e.getValue)(breakOut)
|
val stats: Map[String, Int] = parsed.getStatsList.asScala.iterator.map(e ⇒ e.getKey -> e.getValue).toMap
|
||||||
ShardRegionStats(stats)
|
ShardRegionStats(stats)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -22,6 +22,7 @@ import akka.remote.testkit.MultiNodeSpec
|
||||||
import akka.remote.testkit.STMultiNodeSpec
|
import akka.remote.testkit.STMultiNodeSpec
|
||||||
import akka.remote.transport.ThrottlerTransportAdapter.Direction
|
import akka.remote.transport.ThrottlerTransportAdapter.Direction
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
|
import akka.util.ccompat.imm._
|
||||||
|
|
||||||
object ClusterShardingFailureSpec {
|
object ClusterShardingFailureSpec {
|
||||||
case class Get(id: String)
|
case class Get(id: String)
|
||||||
|
|
@ -125,8 +126,8 @@ abstract class ClusterShardingFailureSpec(config: ClusterShardingFailureSpecConf
|
||||||
|
|
||||||
within(remaining) {
|
within(remaining) {
|
||||||
awaitAssert {
|
awaitAssert {
|
||||||
cluster.state.members.map(_.uniqueAddress) should contain(cluster.selfUniqueAddress)
|
cluster.state.members.unsorted.map(_.uniqueAddress) should contain(cluster.selfUniqueAddress)
|
||||||
cluster.state.members.map(_.status) should ===(Set(MemberStatus.Up))
|
cluster.state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -19,6 +19,7 @@ import org.apache.commons.io.FileUtils
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import akka.cluster.sharding.ShardRegion.GetClusterShardingStats
|
import akka.cluster.sharding.ShardRegion.GetClusterShardingStats
|
||||||
import akka.cluster.sharding.ShardRegion.ClusterShardingStats
|
import akka.cluster.sharding.ShardRegion.ClusterShardingStats
|
||||||
|
import akka.util.ccompat.imm._
|
||||||
|
|
||||||
object ClusterShardingMinMembersSpec {
|
object ClusterShardingMinMembersSpec {
|
||||||
case object StopEntity
|
case object StopEntity
|
||||||
|
|
@ -154,7 +155,7 @@ abstract class ClusterShardingMinMembersSpec(config: ClusterShardingMinMembersSp
|
||||||
within(remaining) {
|
within(remaining) {
|
||||||
awaitAssert {
|
awaitAssert {
|
||||||
cluster.state.members.size should ===(3)
|
cluster.state.members.size should ===(3)
|
||||||
cluster.state.members.map(_.status) should ===(Set(MemberStatus.Up))
|
cluster.state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
enterBarrier("all-up")
|
enterBarrier("all-up")
|
||||||
|
|
|
||||||
|
|
@ -15,6 +15,7 @@ import akka.remote.testkit.{ MultiNodeConfig, MultiNodeSpec, STMultiNodeSpec }
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
import org.apache.commons.io.FileUtils
|
import org.apache.commons.io.FileUtils
|
||||||
|
import akka.util.ccompat.imm._
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
|
|
@ -175,7 +176,7 @@ abstract class ClusterShardingRememberEntitiesSpec(config: ClusterShardingRememb
|
||||||
within(remaining) {
|
within(remaining) {
|
||||||
awaitAssert {
|
awaitAssert {
|
||||||
cluster.state.members.size should ===(2)
|
cluster.state.members.size should ===(2)
|
||||||
cluster.state.members.map(_.status) should ===(Set(MemberStatus.Up))
|
cluster.state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -18,6 +18,7 @@ import akka.remote.testkit.MultiNodeSpec
|
||||||
import akka.remote.testkit.STMultiNodeSpec
|
import akka.remote.testkit.STMultiNodeSpec
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
|
import akka.util.ccompat.imm._
|
||||||
|
|
||||||
object MultiDcClusterShardingSpec {
|
object MultiDcClusterShardingSpec {
|
||||||
sealed trait EntityMsg {
|
sealed trait EntityMsg {
|
||||||
|
|
@ -131,7 +132,7 @@ abstract class MultiDcClusterShardingSpec extends MultiNodeSpec(MultiDcClusterSh
|
||||||
awaitAssert({
|
awaitAssert({
|
||||||
withClue(s"Members: ${Cluster(system).state}") {
|
withClue(s"Members: ${Cluster(system).state}") {
|
||||||
Cluster(system).state.members.size should ===(4)
|
Cluster(system).state.members.size should ===(4)
|
||||||
Cluster(system).state.members.map(_.status) should ===(Set(MemberStatus.Up))
|
Cluster(system).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
|
||||||
}
|
}
|
||||||
}, 10.seconds)
|
}, 10.seconds)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -16,6 +16,7 @@ import akka.testkit.AkkaSpec
|
||||||
import akka.testkit.TestActors.EchoActor
|
import akka.testkit.TestActors.EchoActor
|
||||||
import akka.testkit.TestProbe
|
import akka.testkit.TestProbe
|
||||||
import akka.testkit.WithLogCapturing
|
import akka.testkit.WithLogCapturing
|
||||||
|
import akka.util.ccompat.imm._
|
||||||
|
|
||||||
object CoordinatedShutdownShardingSpec {
|
object CoordinatedShutdownShardingSpec {
|
||||||
val config =
|
val config =
|
||||||
|
|
@ -95,9 +96,9 @@ class CoordinatedShutdownShardingSpec extends AkkaSpec(CoordinatedShutdownShardi
|
||||||
within(10.seconds) {
|
within(10.seconds) {
|
||||||
awaitAssert {
|
awaitAssert {
|
||||||
Cluster(sys1).state.members.size should ===(2)
|
Cluster(sys1).state.members.size should ===(2)
|
||||||
Cluster(sys1).state.members.map(_.status) should ===(Set(MemberStatus.Up))
|
Cluster(sys1).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
|
||||||
Cluster(sys2).state.members.size should ===(2)
|
Cluster(sys2).state.members.size should ===(2)
|
||||||
Cluster(sys2).state.members.map(_.status) should ===(Set(MemberStatus.Up))
|
Cluster(sys2).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -105,11 +106,11 @@ class CoordinatedShutdownShardingSpec extends AkkaSpec(CoordinatedShutdownShardi
|
||||||
within(10.seconds) {
|
within(10.seconds) {
|
||||||
awaitAssert {
|
awaitAssert {
|
||||||
Cluster(sys1).state.members.size should ===(3)
|
Cluster(sys1).state.members.size should ===(3)
|
||||||
Cluster(sys1).state.members.map(_.status) should ===(Set(MemberStatus.Up))
|
Cluster(sys1).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
|
||||||
Cluster(sys2).state.members.size should ===(3)
|
Cluster(sys2).state.members.size should ===(3)
|
||||||
Cluster(sys2).state.members.map(_.status) should ===(Set(MemberStatus.Up))
|
Cluster(sys2).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
|
||||||
Cluster(sys3).state.members.size should ===(3)
|
Cluster(sys3).state.members.size should ===(3)
|
||||||
Cluster(sys3).state.members.map(_.status) should ===(Set(MemberStatus.Up))
|
Cluster(sys3).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -38,6 +38,7 @@ import com.typesafe.config.Config
|
||||||
import akka.remote.DeadlineFailureDetector
|
import akka.remote.DeadlineFailureDetector
|
||||||
import akka.dispatch.Dispatchers
|
import akka.dispatch.Dispatchers
|
||||||
import akka.util.MessageBuffer
|
import akka.util.MessageBuffer
|
||||||
|
import akka.util.ccompat._
|
||||||
import scala.collection.immutable.{ HashMap, HashSet }
|
import scala.collection.immutable.{ HashMap, HashSet }
|
||||||
|
|
||||||
object ClusterClientSettings {
|
object ClusterClientSettings {
|
||||||
|
|
@ -332,7 +333,7 @@ final class ClusterClient(settings: ClusterClientSettings) extends Actor with Ac
|
||||||
val failureDetector = new DeadlineFailureDetector(acceptableHeartbeatPause, heartbeatInterval)
|
val failureDetector = new DeadlineFailureDetector(acceptableHeartbeatPause, heartbeatInterval)
|
||||||
|
|
||||||
var contactPaths: HashSet[ActorPath] =
|
var contactPaths: HashSet[ActorPath] =
|
||||||
initialContacts.to[HashSet]
|
initialContacts.to(HashSet)
|
||||||
val initialContactsSel =
|
val initialContactsSel =
|
||||||
contactPaths.map(context.actorSelection)
|
contactPaths.map(context.actorSelection)
|
||||||
var contacts = initialContactsSel
|
var contacts = initialContactsSel
|
||||||
|
|
@ -373,7 +374,7 @@ final class ClusterClient(settings: ClusterClientSettings) extends Actor with Ac
|
||||||
{
|
{
|
||||||
case Contacts(contactPoints) ⇒
|
case Contacts(contactPoints) ⇒
|
||||||
if (contactPoints.nonEmpty) {
|
if (contactPoints.nonEmpty) {
|
||||||
contactPaths = contactPoints.map(ActorPath.fromString).to[HashSet]
|
contactPaths = contactPoints.map(ActorPath.fromString).to(HashSet)
|
||||||
contacts = contactPaths.map(context.actorSelection)
|
contacts = contactPaths.map(context.actorSelection)
|
||||||
contacts foreach { _ ! Identify(Array.emptyByteArray) }
|
contacts foreach { _ ! Identify(Array.emptyByteArray) }
|
||||||
}
|
}
|
||||||
|
|
@ -423,7 +424,7 @@ final class ClusterClient(settings: ClusterClientSettings) extends Actor with Ac
|
||||||
case Contacts(contactPoints) ⇒
|
case Contacts(contactPoints) ⇒
|
||||||
// refresh of contacts
|
// refresh of contacts
|
||||||
if (contactPoints.nonEmpty) {
|
if (contactPoints.nonEmpty) {
|
||||||
contactPaths = contactPoints.map(ActorPath.fromString).to[HashSet]
|
contactPaths = contactPoints.map(ActorPath.fromString).to(HashSet)
|
||||||
contacts = contactPaths.map(context.actorSelection)
|
contacts = contactPaths.map(context.actorSelection)
|
||||||
}
|
}
|
||||||
publishContactPoints()
|
publishContactPoints()
|
||||||
|
|
@ -951,7 +952,7 @@ final class ClusterReceptionist(pubSubMediator: ActorRef, settings: ClusterRecep
|
||||||
// is the same from all nodes (most of the time) and it also
|
// is the same from all nodes (most of the time) and it also
|
||||||
// load balances the client connections among the nodes in the cluster.
|
// load balances the client connections among the nodes in the cluster.
|
||||||
if (numberOfContacts >= nodes.size) {
|
if (numberOfContacts >= nodes.size) {
|
||||||
val contacts = Contacts(nodes.map(a ⇒ self.path.toStringWithAddress(a))(collection.breakOut))
|
val contacts = Contacts(nodes.iterator.map(a ⇒ self.path.toStringWithAddress(a)).to(immutable.IndexedSeq))
|
||||||
if (log.isDebugEnabled)
|
if (log.isDebugEnabled)
|
||||||
log.debug("Client [{}] gets contactPoints [{}] (all nodes)", sender().path, contacts.contactPoints.mkString(","))
|
log.debug("Client [{}] gets contactPoints [{}] (all nodes)", sender().path, contacts.contactPoints.mkString(","))
|
||||||
sender() ! contacts
|
sender() ! contacts
|
||||||
|
|
@ -960,11 +961,11 @@ final class ClusterReceptionist(pubSubMediator: ActorRef, settings: ClusterRecep
|
||||||
// toStringWithAddress will use the remote address of the client
|
// toStringWithAddress will use the remote address of the client
|
||||||
val a = consistentHash.nodeFor(sender().path.toStringWithAddress(cluster.selfAddress))
|
val a = consistentHash.nodeFor(sender().path.toStringWithAddress(cluster.selfAddress))
|
||||||
val slice = {
|
val slice = {
|
||||||
val first = nodes.from(a).tail.take(numberOfContacts)
|
val first = nodes.rangeFrom(a).tail.take(numberOfContacts)
|
||||||
if (first.size == numberOfContacts) first
|
if (first.size == numberOfContacts) first
|
||||||
else first union nodes.take(numberOfContacts - first.size)
|
else first union nodes.take(numberOfContacts - first.size)
|
||||||
}
|
}
|
||||||
val contacts = Contacts(slice.map(a ⇒ self.path.toStringWithAddress(a))(collection.breakOut))
|
val contacts = Contacts(slice.iterator.map(a ⇒ self.path.toStringWithAddress(a)).to(immutable.IndexedSeq))
|
||||||
if (log.isDebugEnabled)
|
if (log.isDebugEnabled)
|
||||||
log.debug("Client [{}] gets contactPoints [{}]", sender().path, contacts.contactPoints.mkString(","))
|
log.debug("Client [{}] gets contactPoints [{}]", sender().path, contacts.contactPoints.mkString(","))
|
||||||
sender() ! contacts
|
sender() ! contacts
|
||||||
|
|
@ -992,7 +993,7 @@ final class ClusterReceptionist(pubSubMediator: ActorRef, settings: ClusterRecep
|
||||||
|
|
||||||
case SubscribeClusterClients ⇒
|
case SubscribeClusterClients ⇒
|
||||||
val subscriber = sender()
|
val subscriber = sender()
|
||||||
subscriber ! ClusterClients(clientInteractions.keySet.to[HashSet])
|
subscriber ! ClusterClients(clientInteractions.keySet.to(HashSet))
|
||||||
subscribers :+= subscriber
|
subscribers :+= subscriber
|
||||||
context.watch(subscriber)
|
context.watch(subscriber)
|
||||||
|
|
||||||
|
|
@ -1004,7 +1005,7 @@ final class ClusterReceptionist(pubSubMediator: ActorRef, settings: ClusterRecep
|
||||||
self.tell(UnsubscribeClusterClients, subscriber)
|
self.tell(UnsubscribeClusterClients, subscriber)
|
||||||
|
|
||||||
case GetClusterClients ⇒
|
case GetClusterClients ⇒
|
||||||
sender() ! ClusterClients(clientInteractions.keySet.to[HashSet])
|
sender() ! ClusterClients(clientInteractions.keySet.to(HashSet))
|
||||||
|
|
||||||
case CheckDeadlines ⇒
|
case CheckDeadlines ⇒
|
||||||
clientInteractions = clientInteractions.filter {
|
clientInteractions = clientInteractions.filter {
|
||||||
|
|
@ -1025,11 +1026,11 @@ final class ClusterReceptionist(pubSubMediator: ActorRef, settings: ClusterRecep
|
||||||
log.debug("Received new contact from [{}]", client.path)
|
log.debug("Received new contact from [{}]", client.path)
|
||||||
val clusterClientUp = ClusterClientUp(client)
|
val clusterClientUp = ClusterClientUp(client)
|
||||||
subscribers.foreach(_ ! clusterClientUp)
|
subscribers.foreach(_ ! clusterClientUp)
|
||||||
clientsPublished = clientInteractions.keySet.to[HashSet]
|
clientsPublished = clientInteractions.keySet.to(HashSet)
|
||||||
}
|
}
|
||||||
|
|
||||||
def publishClientsUnreachable(): Unit = {
|
def publishClientsUnreachable(): Unit = {
|
||||||
val publishableClients = clientInteractions.keySet.to[HashSet]
|
val publishableClients = clientInteractions.keySet.to(HashSet)
|
||||||
for (c ← clientsPublished if !publishableClients.contains(c)) {
|
for (c ← clientsPublished if !publishableClients.contains(c)) {
|
||||||
log.debug("Lost contact with [{}]", c.path)
|
log.debug("Lost contact with [{}]", c.path)
|
||||||
val clusterClientUnreachable = ClusterClientUnreachable(c)
|
val clusterClientUnreachable = ClusterClientUnreachable(c)
|
||||||
|
|
|
||||||
|
|
@ -569,8 +569,8 @@ class DistributedPubSubMediator(settings: DistributedPubSubSettings) extends Act
|
||||||
case _ ⇒
|
case _ ⇒
|
||||||
(for {
|
(for {
|
||||||
(_, bucket) ← registry
|
(_, bucket) ← registry
|
||||||
valueHolder ← bucket.content.get(path)
|
valueHolder ← bucket.content.get(path).toSeq
|
||||||
routee ← valueHolder.routee
|
routee ← valueHolder.routee.toSeq
|
||||||
} yield routee).toVector
|
} yield routee).toVector
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -751,8 +751,8 @@ class DistributedPubSubMediator(settings: DistributedPubSubSettings) extends Act
|
||||||
val refs = for {
|
val refs = for {
|
||||||
(address, bucket) ← registry
|
(address, bucket) ← registry
|
||||||
if !(allButSelf && address == selfAddress) // if we should skip sender() node and current address == self address => skip
|
if !(allButSelf && address == selfAddress) // if we should skip sender() node and current address == self address => skip
|
||||||
valueHolder ← bucket.content.get(path)
|
valueHolder ← bucket.content.get(path).toSeq
|
||||||
ref ← valueHolder.ref
|
ref ← valueHolder.ref.toSeq
|
||||||
} yield ref
|
} yield ref
|
||||||
if (refs.isEmpty) ignoreOrSendToDeadLetters(msg)
|
if (refs.isEmpty) ignoreOrSendToDeadLetters(msg)
|
||||||
else refs.foreach(_.forward(msg))
|
else refs.foreach(_.forward(msg))
|
||||||
|
|
@ -793,7 +793,7 @@ class DistributedPubSubMediator(settings: DistributedPubSubSettings) extends Act
|
||||||
val topicPrefix = self.path.toStringWithoutAddress
|
val topicPrefix = self.path.toStringWithoutAddress
|
||||||
(for {
|
(for {
|
||||||
(_, bucket) ← registry
|
(_, bucket) ← registry
|
||||||
(key, value) ← bucket.content
|
(key, value) ← bucket.content.toSeq
|
||||||
if key.startsWith(topicPrefix)
|
if key.startsWith(topicPrefix)
|
||||||
topic = key.substring(topicPrefix.length + 1)
|
topic = key.substring(topicPrefix.length + 1)
|
||||||
if !topic.contains('/') // exclude group topics
|
if !topic.contains('/') // exclude group topics
|
||||||
|
|
|
||||||
|
|
@ -5,7 +5,6 @@
|
||||||
package akka.cluster.pubsub.protobuf
|
package akka.cluster.pubsub.protobuf
|
||||||
|
|
||||||
import akka.serialization._
|
import akka.serialization._
|
||||||
import scala.collection.breakOut
|
|
||||||
import akka.actor.{ Address, ExtendedActorSystem }
|
import akka.actor.{ Address, ExtendedActorSystem }
|
||||||
import java.io.{ ByteArrayInputStream, ByteArrayOutputStream }
|
import java.io.{ ByteArrayInputStream, ByteArrayOutputStream }
|
||||||
import akka.protobuf.{ ByteString, MessageLite }
|
import akka.protobuf.{ ByteString, MessageLite }
|
||||||
|
|
@ -17,6 +16,7 @@ import scala.collection.JavaConverters._
|
||||||
import akka.cluster.pubsub.DistributedPubSubMediator._
|
import akka.cluster.pubsub.DistributedPubSubMediator._
|
||||||
import akka.cluster.pubsub.DistributedPubSubMediator.Internal._
|
import akka.cluster.pubsub.DistributedPubSubMediator.Internal._
|
||||||
import akka.actor.ActorRef
|
import akka.actor.ActorRef
|
||||||
|
import akka.util.ccompat._
|
||||||
import scala.collection.immutable.TreeMap
|
import scala.collection.immutable.TreeMap
|
||||||
import java.io.NotSerializableException
|
import java.io.NotSerializableException
|
||||||
|
|
||||||
|
|
@ -127,8 +127,8 @@ private[akka] class DistributedPubSubMessageSerializer(val system: ExtendedActor
|
||||||
|
|
||||||
private def statusFromProto(status: dm.Status): Status = {
|
private def statusFromProto(status: dm.Status): Status = {
|
||||||
val isReplyToStatus = if (status.hasReplyToStatus) status.getReplyToStatus else false
|
val isReplyToStatus = if (status.hasReplyToStatus) status.getReplyToStatus else false
|
||||||
Status(status.getVersionsList.asScala.map(v ⇒
|
Status(status.getVersionsList.asScala.iterator.map(v ⇒
|
||||||
addressFromProto(v.getAddress) → v.getTimestamp)(breakOut), isReplyToStatus)
|
addressFromProto(v.getAddress) → v.getTimestamp).toMap, isReplyToStatus)
|
||||||
}
|
}
|
||||||
|
|
||||||
private def deltaToProto(delta: Delta): dm.Delta = {
|
private def deltaToProto(delta: Delta): dm.Delta = {
|
||||||
|
|
@ -154,9 +154,9 @@ private[akka] class DistributedPubSubMessageSerializer(val system: ExtendedActor
|
||||||
|
|
||||||
private def deltaFromProto(delta: dm.Delta): Delta =
|
private def deltaFromProto(delta: dm.Delta): Delta =
|
||||||
Delta(delta.getBucketsList.asScala.toVector.map { b ⇒
|
Delta(delta.getBucketsList.asScala.toVector.map { b ⇒
|
||||||
val content: TreeMap[String, ValueHolder] = b.getContentList.asScala.map { entry ⇒
|
val content: TreeMap[String, ValueHolder] = scala.collection.immutable.TreeMap.from(b.getContentList.asScala.iterator.map { entry ⇒
|
||||||
entry.getKey → ValueHolder(entry.getVersion, if (entry.hasRef) Some(resolveActorRef(entry.getRef)) else None)
|
entry.getKey → ValueHolder(entry.getVersion, if (entry.hasRef) Some(resolveActorRef(entry.getRef)) else None)
|
||||||
}(breakOut)
|
})
|
||||||
Bucket(addressFromProto(b.getOwner), b.getVersion, content)
|
Bucket(addressFromProto(b.getOwner), b.getVersion, content)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -70,6 +70,7 @@ class ClusterSingletonLeavingSpeedSpec extends AkkaSpec("""
|
||||||
override def expectedTestDuration: FiniteDuration = 10.minutes
|
override def expectedTestDuration: FiniteDuration = 10.minutes
|
||||||
|
|
||||||
def join(from: ActorSystem, to: ActorSystem, probe: ActorRef): Unit = {
|
def join(from: ActorSystem, to: ActorSystem, probe: ActorRef): Unit = {
|
||||||
|
|
||||||
from.actorOf(
|
from.actorOf(
|
||||||
ClusterSingletonManager.props(
|
ClusterSingletonManager.props(
|
||||||
singletonProps = TheSingleton.props(probe),
|
singletonProps = TheSingleton.props(probe),
|
||||||
|
|
@ -79,9 +80,10 @@ class ClusterSingletonLeavingSpeedSpec extends AkkaSpec("""
|
||||||
|
|
||||||
Cluster(from).join(Cluster(to).selfAddress)
|
Cluster(from).join(Cluster(to).selfAddress)
|
||||||
within(15.seconds) {
|
within(15.seconds) {
|
||||||
|
import akka.util.ccompat.imm._
|
||||||
awaitAssert {
|
awaitAssert {
|
||||||
Cluster(from).state.members.map(_.uniqueAddress) should contain(Cluster(from).selfUniqueAddress)
|
Cluster(from).state.members.map(_.uniqueAddress) should contain(Cluster(from).selfUniqueAddress)
|
||||||
Cluster(from).state.members.map(_.status) should ===(Set(MemberStatus.Up))
|
Cluster(from).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -62,10 +62,11 @@ class ClusterSingletonRestart2Spec extends AkkaSpec("""
|
||||||
name = "echo")
|
name = "echo")
|
||||||
|
|
||||||
within(45.seconds) {
|
within(45.seconds) {
|
||||||
|
import akka.util.ccompat.imm._
|
||||||
awaitAssert {
|
awaitAssert {
|
||||||
Cluster(from) join Cluster(to).selfAddress
|
Cluster(from) join Cluster(to).selfAddress
|
||||||
Cluster(from).state.members.map(_.uniqueAddress) should contain(Cluster(from).selfUniqueAddress)
|
Cluster(from).state.members.map(_.uniqueAddress) should contain(Cluster(from).selfUniqueAddress)
|
||||||
Cluster(from).state.members.map(_.status) should ===(Set(MemberStatus.Up))
|
Cluster(from).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -43,10 +43,11 @@ class ClusterSingletonRestartSpec extends AkkaSpec("""
|
||||||
name = "echo")
|
name = "echo")
|
||||||
|
|
||||||
within(10.seconds) {
|
within(10.seconds) {
|
||||||
|
import akka.util.ccompat.imm._
|
||||||
awaitAssert {
|
awaitAssert {
|
||||||
Cluster(from) join Cluster(to).selfAddress
|
Cluster(from) join Cluster(to).selfAddress
|
||||||
Cluster(from).state.members.map(_.uniqueAddress) should contain(Cluster(from).selfUniqueAddress)
|
Cluster(from).state.members.map(_.uniqueAddress) should contain(Cluster(from).selfUniqueAddress)
|
||||||
Cluster(from).state.members.map(_.status) should ===(Set(MemberStatus.Up))
|
Cluster(from).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -16,7 +16,7 @@ import akka.dispatch.MonitorableThreadFactory
|
||||||
import akka.event.{ Logging, LoggingAdapter }
|
import akka.event.{ Logging, LoggingAdapter }
|
||||||
import akka.japi.Util
|
import akka.japi.Util
|
||||||
import akka.pattern._
|
import akka.pattern._
|
||||||
import akka.remote.{ DefaultFailureDetectorRegistry, _ }
|
import akka.remote.{ UniqueAddress ⇒ _, _ }
|
||||||
import com.typesafe.config.{ Config, ConfigFactory }
|
import com.typesafe.config.{ Config, ConfigFactory }
|
||||||
|
|
||||||
import scala.annotation.varargs
|
import scala.annotation.varargs
|
||||||
|
|
|
||||||
|
|
@ -1258,9 +1258,7 @@ private[cluster] class ClusterCoreDaemon(publisher: ActorRef, joinConfigCompatCh
|
||||||
}
|
}
|
||||||
|
|
||||||
if (changedMembers.nonEmpty) {
|
if (changedMembers.nonEmpty) {
|
||||||
// replace changed members
|
val newGossip = localGossip.update(changedMembers)
|
||||||
val newMembers = changedMembers union localMembers
|
|
||||||
val newGossip = localGossip.copy(members = newMembers)
|
|
||||||
updateLatestGossip(newGossip)
|
updateLatestGossip(newGossip)
|
||||||
|
|
||||||
// log status changes
|
// log status changes
|
||||||
|
|
|
||||||
|
|
@ -15,8 +15,8 @@ import akka.event.EventStream
|
||||||
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
|
import akka.dispatch.{ RequiresMessageQueue, UnboundedMessageQueueSemantics }
|
||||||
import akka.actor.DeadLetterSuppression
|
import akka.actor.DeadLetterSuppression
|
||||||
import akka.annotation.{ DoNotInherit, InternalApi }
|
import akka.annotation.{ DoNotInherit, InternalApi }
|
||||||
|
import akka.util.ccompat._
|
||||||
|
|
||||||
import scala.collection.breakOut
|
|
||||||
import scala.runtime.AbstractFunction5
|
import scala.runtime.AbstractFunction5
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -162,7 +162,7 @@ object ClusterEvent {
|
||||||
/**
|
/**
|
||||||
* All data centers in the cluster
|
* All data centers in the cluster
|
||||||
*/
|
*/
|
||||||
def allDataCenters: Set[String] = members.map(_.dataCenter)(breakOut)
|
def allDataCenters: Set[String] = members.iterator.map(_.dataCenter).to(immutable.Set)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API: All data centers in the cluster
|
* Java API: All data centers in the cluster
|
||||||
|
|
@ -380,10 +380,10 @@ object ClusterEvent {
|
||||||
else {
|
else {
|
||||||
val newGossip = newState.latestGossip
|
val newGossip = newState.latestGossip
|
||||||
val oldUnreachableNodes = oldState.dcReachabilityNoOutsideNodes.allUnreachableOrTerminated
|
val oldUnreachableNodes = oldState.dcReachabilityNoOutsideNodes.allUnreachableOrTerminated
|
||||||
newState.dcReachabilityNoOutsideNodes.allUnreachableOrTerminated.collect {
|
newState.dcReachabilityNoOutsideNodes.allUnreachableOrTerminated.iterator.collect {
|
||||||
case node if !oldUnreachableNodes.contains(node) && node != newState.selfUniqueAddress ⇒
|
case node if !oldUnreachableNodes.contains(node) && node != newState.selfUniqueAddress ⇒
|
||||||
UnreachableMember(newGossip.member(node))
|
UnreachableMember(newGossip.member(node))
|
||||||
}(collection.breakOut)
|
}.to(immutable.IndexedSeq)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -393,10 +393,10 @@ object ClusterEvent {
|
||||||
if (newState eq oldState) Nil
|
if (newState eq oldState) Nil
|
||||||
else {
|
else {
|
||||||
val newGossip = newState.latestGossip
|
val newGossip = newState.latestGossip
|
||||||
oldState.dcReachabilityNoOutsideNodes.allUnreachable.collect {
|
oldState.dcReachabilityNoOutsideNodes.allUnreachable.iterator.collect {
|
||||||
case node if newGossip.hasMember(node) && newState.dcReachabilityNoOutsideNodes.isReachable(node) && node != newState.selfUniqueAddress ⇒
|
case node if newGossip.hasMember(node) && newState.dcReachabilityNoOutsideNodes.isReachable(node) && node != newState.selfUniqueAddress ⇒
|
||||||
ReachableMember(newGossip.member(node))
|
ReachableMember(newGossip.member(node))
|
||||||
}(collection.breakOut)
|
}.to(immutable.IndexedSeq)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -418,7 +418,7 @@ object ClusterEvent {
|
||||||
if (newState eq oldState) Nil
|
if (newState eq oldState) Nil
|
||||||
else {
|
else {
|
||||||
val otherDcs = (oldState.latestGossip.allDataCenters union newState.latestGossip.allDataCenters) - newState.selfDc
|
val otherDcs = (oldState.latestGossip.allDataCenters union newState.latestGossip.allDataCenters) - newState.selfDc
|
||||||
otherDcs.filterNot(isReachable(newState, oldState.dcReachability.allUnreachableOrTerminated)).map(UnreachableDataCenter)(collection.breakOut)
|
otherDcs.filterNot(isReachable(newState, oldState.dcReachability.allUnreachableOrTerminated)).iterator.map(UnreachableDataCenter).to(immutable.IndexedSeq)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -433,7 +433,7 @@ object ClusterEvent {
|
||||||
val oldUnreachableDcs = otherDcs.filterNot(isReachable(oldState, Set()))
|
val oldUnreachableDcs = otherDcs.filterNot(isReachable(oldState, Set()))
|
||||||
val currentUnreachableDcs = otherDcs.filterNot(isReachable(newState, Set()))
|
val currentUnreachableDcs = otherDcs.filterNot(isReachable(newState, Set()))
|
||||||
|
|
||||||
(oldUnreachableDcs diff currentUnreachableDcs).map(ReachableDataCenter)(collection.breakOut)
|
(oldUnreachableDcs diff currentUnreachableDcs).iterator.map(ReachableDataCenter).to(immutable.IndexedSeq)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -451,7 +451,8 @@ object ClusterEvent {
|
||||||
case (_, newMember :: oldMember :: Nil) if newMember.status != oldMember.status || newMember.upNumber != oldMember.upNumber ⇒
|
case (_, newMember :: oldMember :: Nil) if newMember.status != oldMember.status || newMember.upNumber != oldMember.upNumber ⇒
|
||||||
newMember
|
newMember
|
||||||
}
|
}
|
||||||
val memberEvents = (newMembers ++ changedMembers) collect {
|
import akka.util.ccompat.imm._
|
||||||
|
val memberEvents = (newMembers ++ changedMembers).unsorted collect {
|
||||||
case m if m.status == Joining ⇒ MemberJoined(m)
|
case m if m.status == Joining ⇒ MemberJoined(m)
|
||||||
case m if m.status == WeaklyUp ⇒ MemberWeaklyUp(m)
|
case m if m.status == WeaklyUp ⇒ MemberWeaklyUp(m)
|
||||||
case m if m.status == Up ⇒ MemberUp(m)
|
case m if m.status == Up ⇒ MemberUp(m)
|
||||||
|
|
@ -462,7 +463,7 @@ object ClusterEvent {
|
||||||
}
|
}
|
||||||
|
|
||||||
val removedMembers = oldGossip.members diff newGossip.members
|
val removedMembers = oldGossip.members diff newGossip.members
|
||||||
val removedEvents = removedMembers.map(m ⇒ MemberRemoved(m.copy(status = Removed), m.status))
|
val removedEvents = removedMembers.unsorted.map(m ⇒ MemberRemoved(m.copy(status = Removed), m.status))
|
||||||
|
|
||||||
(new VectorBuilder[MemberEvent]() ++= removedEvents ++= memberEvents).result()
|
(new VectorBuilder[MemberEvent]() ++= removedEvents ++= memberEvents).result()
|
||||||
}
|
}
|
||||||
|
|
@ -572,8 +573,8 @@ private[cluster] final class ClusterDomainEventPublisher extends Actor with Acto
|
||||||
unreachable = unreachable,
|
unreachable = unreachable,
|
||||||
seenBy = membershipState.latestGossip.seenBy.map(_.address),
|
seenBy = membershipState.latestGossip.seenBy.map(_.address),
|
||||||
leader = membershipState.leader.map(_.address),
|
leader = membershipState.leader.map(_.address),
|
||||||
roleLeaderMap = membershipState.latestGossip.allRoles.map(r ⇒
|
roleLeaderMap = membershipState.latestGossip.allRoles.iterator.map(r ⇒
|
||||||
r → membershipState.roleLeader(r).map(_.address))(collection.breakOut),
|
r → membershipState.roleLeader(r).map(_.address)).toMap,
|
||||||
unreachableDataCenters)
|
unreachableDataCenters)
|
||||||
receiver ! state
|
receiver ! state
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -14,6 +14,7 @@ import akka.cluster.ClusterEvent._
|
||||||
import akka.remote.FailureDetectorRegistry
|
import akka.remote.FailureDetectorRegistry
|
||||||
import akka.remote.HeartbeatMessage
|
import akka.remote.HeartbeatMessage
|
||||||
import akka.annotation.InternalApi
|
import akka.annotation.InternalApi
|
||||||
|
import akka.util.ccompat._
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API.
|
* INTERNAL API.
|
||||||
|
|
@ -338,13 +339,13 @@ private[cluster] final case class HeartbeatNodeRing(
|
||||||
take(n - 1, iter, acc + next) // include the reachable
|
take(n - 1, iter, acc + next) // include the reachable
|
||||||
}
|
}
|
||||||
|
|
||||||
val (remaining, slice1) = take(monitoredByNrOfMembers, nodeRing.from(sender).tail.iterator, Set.empty)
|
val (remaining, slice1) = take(monitoredByNrOfMembers, nodeRing.rangeFrom(sender).tail.iterator, Set.empty)
|
||||||
val slice =
|
val slice =
|
||||||
if (remaining == 0)
|
if (remaining == 0)
|
||||||
slice1
|
slice1
|
||||||
else {
|
else {
|
||||||
// wrap around
|
// wrap around
|
||||||
val (_, slice2) = take(remaining, nodeRing.to(sender).iterator.filterNot(_ == sender), slice1)
|
val (_, slice2) = take(remaining, nodeRing.rangeTo(sender).iterator.filterNot(_ == sender), slice1)
|
||||||
slice2
|
slice2
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -10,8 +10,10 @@ import akka.cluster.ClusterEvent._
|
||||||
import akka.cluster.ClusterSettings.DataCenter
|
import akka.cluster.ClusterSettings.DataCenter
|
||||||
import akka.remote.FailureDetectorRegistry
|
import akka.remote.FailureDetectorRegistry
|
||||||
import akka.util.ConstantFun
|
import akka.util.ConstantFun
|
||||||
|
import akka.util.ccompat._
|
||||||
|
|
||||||
import scala.collection.{ SortedSet, breakOut }
|
import scala.collection.SortedSet
|
||||||
|
import scala.collection.immutable
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
@ -270,8 +272,8 @@ private[cluster] final case class CrossDcHeartbeatingState(
|
||||||
val allOtherNodes = otherDcs.values
|
val allOtherNodes = otherDcs.values
|
||||||
|
|
||||||
allOtherNodes.flatMap(
|
allOtherNodes.flatMap(
|
||||||
_.take(nrOfMonitoredNodesPerDc)
|
_.take(nrOfMonitoredNodesPerDc).iterator
|
||||||
.map(_.uniqueAddress)(breakOut)).toSet
|
.map(_.uniqueAddress).to(immutable.IndexedSeq)).toSet
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Lists addresses in diven DataCenter that this node should send heartbeats to */
|
/** Lists addresses in diven DataCenter that this node should send heartbeats to */
|
||||||
|
|
@ -280,8 +282,8 @@ private[cluster] final case class CrossDcHeartbeatingState(
|
||||||
else {
|
else {
|
||||||
val otherNodes = state.getOrElse(dc, emptyMembersSortedSet)
|
val otherNodes = state.getOrElse(dc, emptyMembersSortedSet)
|
||||||
otherNodes
|
otherNodes
|
||||||
.take(nrOfMonitoredNodesPerDc)
|
.take(nrOfMonitoredNodesPerDc).iterator
|
||||||
.map(_.uniqueAddress)(breakOut)
|
.map(_.uniqueAddress).to(immutable.Set)
|
||||||
}
|
}
|
||||||
|
|
||||||
def allMembers: Iterable[Member] =
|
def allMembers: Iterable[Member] =
|
||||||
|
|
|
||||||
|
|
@ -87,7 +87,7 @@ private[cluster] final case class Gossip(
|
||||||
}
|
}
|
||||||
|
|
||||||
@transient private lazy val membersMap: Map[UniqueAddress, Member] =
|
@transient private lazy val membersMap: Map[UniqueAddress, Member] =
|
||||||
members.map(m ⇒ m.uniqueAddress → m)(collection.breakOut)
|
members.iterator.map(m ⇒ m.uniqueAddress → m).toMap
|
||||||
|
|
||||||
@transient lazy val isMultiDc =
|
@transient lazy val isMultiDc =
|
||||||
if (members.size <= 1) false
|
if (members.size <= 1) false
|
||||||
|
|
@ -209,7 +209,7 @@ private[cluster] final case class Gossip(
|
||||||
}
|
}
|
||||||
|
|
||||||
def update(updatedMembers: immutable.SortedSet[Member]): Gossip = {
|
def update(updatedMembers: immutable.SortedSet[Member]): Gossip = {
|
||||||
copy(members = updatedMembers union members)
|
copy(members = updatedMembers union (members diff updatedMembers))
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -8,6 +8,7 @@ import java.util
|
||||||
|
|
||||||
import akka.actor.ExtendedActorSystem
|
import akka.actor.ExtendedActorSystem
|
||||||
import akka.annotation.{ DoNotInherit, InternalApi }
|
import akka.annotation.{ DoNotInherit, InternalApi }
|
||||||
|
import akka.util.ccompat._
|
||||||
import com.typesafe.config.{ Config, ConfigFactory, ConfigValue }
|
import com.typesafe.config.{ Config, ConfigFactory, ConfigValue }
|
||||||
|
|
||||||
import scala.collection.JavaConverters._
|
import scala.collection.JavaConverters._
|
||||||
|
|
@ -48,7 +49,7 @@ object JoinConfigCompatChecker {
|
||||||
}
|
}
|
||||||
|
|
||||||
if (result.isEmpty) Valid
|
if (result.isEmpty) Valid
|
||||||
else Invalid(result.to[im.Seq])
|
else Invalid(result.to(im.Seq))
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -78,7 +79,7 @@ object JoinConfigCompatChecker {
|
||||||
}
|
}
|
||||||
|
|
||||||
if (incompatibleKeys.isEmpty) Valid
|
if (incompatibleKeys.isEmpty) Valid
|
||||||
else Invalid(incompatibleKeys.to[im.Seq])
|
else Invalid(incompatibleKeys.to(im.Seq))
|
||||||
}
|
}
|
||||||
|
|
||||||
exists(requiredKeys, toCheck) ++ checkEquality
|
exists(requiredKeys, toCheck) ++ checkEquality
|
||||||
|
|
@ -123,7 +124,7 @@ object JoinConfigCompatChecker {
|
||||||
*/
|
*/
|
||||||
@InternalApi
|
@InternalApi
|
||||||
private[cluster] def removeSensitiveKeys(config: Config, clusterSettings: ClusterSettings): im.Seq[String] = {
|
private[cluster] def removeSensitiveKeys(config: Config, clusterSettings: ClusterSettings): im.Seq[String] = {
|
||||||
val existingKeys = config.entrySet().asScala.map(_.getKey).to[im.Seq]
|
val existingKeys = config.entrySet().asScala.map(_.getKey).to(im.Seq)
|
||||||
removeSensitiveKeys(existingKeys, clusterSettings)
|
removeSensitiveKeys(existingKeys, clusterSettings)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -147,7 +148,7 @@ object JoinConfigCompatChecker {
|
||||||
new JoinConfigCompatChecker {
|
new JoinConfigCompatChecker {
|
||||||
override val requiredKeys: im.Seq[String] = {
|
override val requiredKeys: im.Seq[String] = {
|
||||||
// Always include akka.version (used in join logging)
|
// Always include akka.version (used in join logging)
|
||||||
"akka.version" +: checkers.flatMap(_.requiredKeys).to[im.Seq]
|
"akka.version" +: checkers.flatMap(_.requiredKeys).to(im.Seq)
|
||||||
}
|
}
|
||||||
override def check(toValidate: Config, clusterConfig: Config): ConfigValidation =
|
override def check(toValidate: Config, clusterConfig: Config): ConfigValidation =
|
||||||
checkers.foldLeft(Valid: ConfigValidation) { (acc, checker) ⇒
|
checkers.foldLeft(Valid: ConfigValidation) { (acc, checker) ⇒
|
||||||
|
|
|
||||||
|
|
@ -11,9 +11,9 @@ import scala.collection.SortedSet
|
||||||
import akka.cluster.ClusterSettings.DataCenter
|
import akka.cluster.ClusterSettings.DataCenter
|
||||||
import akka.cluster.MemberStatus._
|
import akka.cluster.MemberStatus._
|
||||||
import akka.annotation.InternalApi
|
import akka.annotation.InternalApi
|
||||||
|
import akka.util.ccompat._
|
||||||
|
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
import scala.collection.breakOut
|
|
||||||
import scala.util.Random
|
import scala.util.Random
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -275,10 +275,10 @@ import scala.util.Random
|
||||||
if (preferNodesWithDifferentView(state)) {
|
if (preferNodesWithDifferentView(state)) {
|
||||||
// If it's time to try to gossip to some nodes with a different view
|
// If it's time to try to gossip to some nodes with a different view
|
||||||
// gossip to a random alive same dc member with preference to a member with older gossip version
|
// gossip to a random alive same dc member with preference to a member with older gossip version
|
||||||
latestGossip.members.collect {
|
latestGossip.members.iterator.collect {
|
||||||
case m if m.dataCenter == state.selfDc && !latestGossip.seenByNode(m.uniqueAddress) && state.validNodeForGossip(m.uniqueAddress) ⇒
|
case m if m.dataCenter == state.selfDc && !latestGossip.seenByNode(m.uniqueAddress) && state.validNodeForGossip(m.uniqueAddress) ⇒
|
||||||
m.uniqueAddress
|
m.uniqueAddress
|
||||||
}(breakOut)
|
}.to(Vector)
|
||||||
} else Vector.empty
|
} else Vector.empty
|
||||||
|
|
||||||
// Fall back to localGossip
|
// Fall back to localGossip
|
||||||
|
|
|
||||||
|
|
@ -7,7 +7,7 @@ package akka.cluster
|
||||||
import akka.annotation.InternalApi
|
import akka.annotation.InternalApi
|
||||||
|
|
||||||
import scala.collection.immutable
|
import scala.collection.immutable
|
||||||
import scala.collection.breakOut
|
import akka.util.ccompat._
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
@ -72,9 +72,8 @@ private[cluster] class Reachability private (
|
||||||
(observerRowsMap, allUnreachable, allTerminated)
|
(observerRowsMap, allUnreachable, allTerminated)
|
||||||
} else {
|
} else {
|
||||||
val mapBuilder = scala.collection.mutable.Map.empty[UniqueAddress, Map[UniqueAddress, Reachability.Record]]
|
val mapBuilder = scala.collection.mutable.Map.empty[UniqueAddress, Map[UniqueAddress, Reachability.Record]]
|
||||||
import scala.collection.mutable.SetBuilder
|
var allTerminated = Set.empty[UniqueAddress]
|
||||||
val terminatedBuilder = new SetBuilder[UniqueAddress, Set[UniqueAddress]](Set.empty)
|
var allUnreachable = Set.empty[UniqueAddress]
|
||||||
val unreachableBuilder = new SetBuilder[UniqueAddress, Set[UniqueAddress]](Set.empty)
|
|
||||||
|
|
||||||
records foreach { r ⇒
|
records foreach { r ⇒
|
||||||
val m = mapBuilder.get(r.observer) match {
|
val m = mapBuilder.get(r.observer) match {
|
||||||
|
|
@ -83,15 +82,13 @@ private[cluster] class Reachability private (
|
||||||
}
|
}
|
||||||
mapBuilder += (r.observer → m)
|
mapBuilder += (r.observer → m)
|
||||||
|
|
||||||
if (r.status == Unreachable) unreachableBuilder += r.subject
|
if (r.status == Unreachable) allUnreachable += r.subject
|
||||||
else if (r.status == Terminated) terminatedBuilder += r.subject
|
else if (r.status == Terminated) allTerminated += r.subject
|
||||||
}
|
}
|
||||||
|
|
||||||
val observerRowsMap: Map[UniqueAddress, Map[UniqueAddress, Reachability.Record]] = mapBuilder.toMap
|
val observerRowsMap: Map[UniqueAddress, Map[UniqueAddress, Reachability.Record]] = mapBuilder.toMap
|
||||||
val allTerminated: Set[UniqueAddress] = terminatedBuilder.result()
|
|
||||||
val allUnreachable: Set[UniqueAddress] = unreachableBuilder.result() diff allTerminated
|
|
||||||
|
|
||||||
(observerRowsMap, allUnreachable, allTerminated)
|
(observerRowsMap, allUnreachable diff allTerminated, allTerminated)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -195,7 +192,7 @@ private[cluster] class Reachability private (
|
||||||
}
|
}
|
||||||
|
|
||||||
def remove(nodes: Iterable[UniqueAddress]): Reachability = {
|
def remove(nodes: Iterable[UniqueAddress]): Reachability = {
|
||||||
val nodesSet = nodes.to[immutable.HashSet]
|
val nodesSet = nodes.to(immutable.HashSet)
|
||||||
val newRecords = records.filterNot(r ⇒ nodesSet(r.observer) || nodesSet(r.subject))
|
val newRecords = records.filterNot(r ⇒ nodesSet(r.observer) || nodesSet(r.subject))
|
||||||
val newVersions = versions -- nodes
|
val newVersions = versions -- nodes
|
||||||
Reachability(newRecords, newVersions)
|
Reachability(newRecords, newVersions)
|
||||||
|
|
@ -265,16 +262,16 @@ private[cluster] class Reachability private (
|
||||||
observerRows(observer) match {
|
observerRows(observer) match {
|
||||||
case None ⇒ Set.empty
|
case None ⇒ Set.empty
|
||||||
case Some(observerRows) ⇒
|
case Some(observerRows) ⇒
|
||||||
observerRows.collect {
|
observerRows.iterator.collect {
|
||||||
case (subject, record) if record.status == Unreachable ⇒ subject
|
case (subject, record) if record.status == Unreachable ⇒ subject
|
||||||
}(breakOut)
|
}.to(immutable.Set)
|
||||||
}
|
}
|
||||||
|
|
||||||
def observersGroupedByUnreachable: Map[UniqueAddress, Set[UniqueAddress]] = {
|
def observersGroupedByUnreachable: Map[UniqueAddress, Set[UniqueAddress]] = {
|
||||||
records.groupBy(_.subject).collect {
|
records.groupBy(_.subject).collect {
|
||||||
case (subject, records) if records.exists(_.status == Unreachable) ⇒
|
case (subject, records) if records.exists(_.status == Unreachable) ⇒
|
||||||
val observers: Set[UniqueAddress] =
|
val observers: Set[UniqueAddress] =
|
||||||
records.collect { case r if r.status == Unreachable ⇒ r.observer }(breakOut)
|
records.iterator.collect { case r if r.status == Unreachable ⇒ r.observer }.to(immutable.Set)
|
||||||
(subject → observers)
|
(subject → observers)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -21,6 +21,8 @@ import akka.annotation.InternalApi
|
||||||
import akka.cluster.InternalClusterAction._
|
import akka.cluster.InternalClusterAction._
|
||||||
import akka.cluster.routing.{ ClusterRouterPool, ClusterRouterPoolSettings }
|
import akka.cluster.routing.{ ClusterRouterPool, ClusterRouterPoolSettings }
|
||||||
import akka.routing.Pool
|
import akka.routing.Pool
|
||||||
|
import akka.util.ccompat._
|
||||||
|
import akka.util.ccompat.imm._
|
||||||
import com.typesafe.config.{ Config, ConfigFactory, ConfigRenderOptions }
|
import com.typesafe.config.{ Config, ConfigFactory, ConfigRenderOptions }
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -370,9 +372,9 @@ final class ClusterMessageSerializer(val system: ExtendedActorSystem) extends Se
|
||||||
val allMembers = gossip.members.toVector
|
val allMembers = gossip.members.toVector
|
||||||
val allAddresses: Vector[UniqueAddress] = allMembers.map(_.uniqueAddress) ++ gossip.tombstones.keys
|
val allAddresses: Vector[UniqueAddress] = allMembers.map(_.uniqueAddress) ++ gossip.tombstones.keys
|
||||||
val addressMapping = allAddresses.zipWithIndex.toMap
|
val addressMapping = allAddresses.zipWithIndex.toMap
|
||||||
val allRoles = allMembers.foldLeft(Set.empty[String])((acc, m) ⇒ acc union m.roles).to[Vector]
|
val allRoles = allMembers.foldLeft(Set.empty[String])((acc, m) ⇒ acc union m.roles).to(Vector)
|
||||||
val roleMapping = allRoles.zipWithIndex.toMap
|
val roleMapping = allRoles.zipWithIndex.toMap
|
||||||
val allHashes = gossip.version.versions.keys.to[Vector]
|
val allHashes = gossip.version.versions.keys.to(Vector)
|
||||||
val hashMapping = allHashes.zipWithIndex.toMap
|
val hashMapping = allHashes.zipWithIndex.toMap
|
||||||
|
|
||||||
def mapUniqueAddress(uniqueAddress: UniqueAddress): Integer = mapWithErrorMessage(addressMapping, uniqueAddress, "address")
|
def mapUniqueAddress(uniqueAddress: UniqueAddress): Integer = mapWithErrorMessage(addressMapping, uniqueAddress, "address")
|
||||||
|
|
@ -403,7 +405,7 @@ final class ClusterMessageSerializer(val system: ExtendedActorSystem) extends Se
|
||||||
.build()
|
.build()
|
||||||
|
|
||||||
val reachability = reachabilityToProto(gossip.overview.reachability)
|
val reachability = reachabilityToProto(gossip.overview.reachability)
|
||||||
val members = (gossip.members: Set[Member] /* 2.13.0-M5 change cast to .unsorted */ ).map(memberToProto _)
|
val members = gossip.members.unsorted.map(memberToProto _)
|
||||||
val seen = gossip.overview.seen.map(mapUniqueAddress)
|
val seen = gossip.overview.seen.map(mapUniqueAddress)
|
||||||
|
|
||||||
val overview = cm.GossipOverview.newBuilder.addAllSeen(seen.asJava).
|
val overview = cm.GossipOverview.newBuilder.addAllSeen(seen.asJava).
|
||||||
|
|
@ -447,11 +449,10 @@ final class ClusterMessageSerializer(val system: ExtendedActorSystem) extends Se
|
||||||
gossipStatusFromProto(cm.GossipStatus.parseFrom(bytes))
|
gossipStatusFromProto(cm.GossipStatus.parseFrom(bytes))
|
||||||
|
|
||||||
private def gossipFromProto(gossip: cm.Gossip): Gossip = {
|
private def gossipFromProto(gossip: cm.Gossip): Gossip = {
|
||||||
import scala.collection.breakOut
|
|
||||||
val addressMapping: Vector[UniqueAddress] =
|
val addressMapping: Vector[UniqueAddress] =
|
||||||
gossip.getAllAddressesList.asScala.map(uniqueAddressFromProto)(breakOut)
|
gossip.getAllAddressesList.asScala.iterator.map(uniqueAddressFromProto).to(immutable.Vector)
|
||||||
val roleMapping: Vector[String] = gossip.getAllRolesList.asScala.map(identity)(breakOut)
|
val roleMapping: Vector[String] = gossip.getAllRolesList.asScala.iterator.map(identity).to(immutable.Vector)
|
||||||
val hashMapping: Vector[String] = gossip.getAllHashesList.asScala.map(identity)(breakOut)
|
val hashMapping: Vector[String] = gossip.getAllHashesList.asScala.iterator.map(identity).to(immutable.Vector)
|
||||||
|
|
||||||
def reachabilityFromProto(observerReachability: Iterable[cm.ObserverReachability]): Reachability = {
|
def reachabilityFromProto(observerReachability: Iterable[cm.ObserverReachability]): Reachability = {
|
||||||
val recordBuilder = new immutable.VectorBuilder[Reachability.Record]
|
val recordBuilder = new immutable.VectorBuilder[Reachability.Record]
|
||||||
|
|
@ -492,20 +493,19 @@ final class ClusterMessageSerializer(val system: ExtendedActorSystem) extends Se
|
||||||
def tombstoneFromProto(tombstone: cm.Tombstone): (UniqueAddress, Long) =
|
def tombstoneFromProto(tombstone: cm.Tombstone): (UniqueAddress, Long) =
|
||||||
(addressMapping(tombstone.getAddressIndex), tombstone.getTimestamp)
|
(addressMapping(tombstone.getAddressIndex), tombstone.getTimestamp)
|
||||||
|
|
||||||
val members: immutable.SortedSet[Member] = gossip.getMembersList.asScala.map(memberFromProto)(breakOut)
|
val members: immutable.SortedSet[Member] = gossip.getMembersList.asScala.iterator.map(memberFromProto).to(immutable.SortedSet)
|
||||||
|
|
||||||
val reachability = reachabilityFromProto(gossip.getOverview.getObserverReachabilityList.asScala)
|
val reachability = reachabilityFromProto(gossip.getOverview.getObserverReachabilityList.asScala)
|
||||||
val seen: Set[UniqueAddress] = gossip.getOverview.getSeenList.asScala.map(addressMapping(_))(breakOut)
|
val seen: Set[UniqueAddress] = gossip.getOverview.getSeenList.asScala.iterator.map(addressMapping(_)).to(immutable.Set)
|
||||||
val overview = GossipOverview(seen, reachability)
|
val overview = GossipOverview(seen, reachability)
|
||||||
val tombstones: Map[UniqueAddress, Long] = gossip.getTombstonesList.asScala.map(tombstoneFromProto)(breakOut)
|
val tombstones: Map[UniqueAddress, Long] = gossip.getTombstonesList.asScala.iterator.map(tombstoneFromProto).toMap
|
||||||
|
|
||||||
Gossip(members, overview, vectorClockFromProto(gossip.getVersion, hashMapping), tombstones)
|
Gossip(members, overview, vectorClockFromProto(gossip.getVersion, hashMapping), tombstones)
|
||||||
}
|
}
|
||||||
|
|
||||||
private def vectorClockFromProto(version: cm.VectorClock, hashMapping: immutable.Seq[String]) = {
|
private def vectorClockFromProto(version: cm.VectorClock, hashMapping: immutable.Seq[String]) = {
|
||||||
import scala.collection.breakOut
|
VectorClock(scala.collection.immutable.TreeMap.from(version.getVersionsList.asScala.iterator.map(
|
||||||
VectorClock(version.getVersionsList.asScala.map(
|
v ⇒ (VectorClock.Node.fromHash(hashMapping(v.getHashIndex)), v.getTimestamp))))
|
||||||
v ⇒ (VectorClock.Node.fromHash(hashMapping(v.getHashIndex)), v.getTimestamp))(breakOut))
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private def gossipEnvelopeFromProto(envelope: cm.GossipEnvelope): GossipEnvelope = {
|
private def gossipEnvelopeFromProto(envelope: cm.GossipEnvelope): GossipEnvelope = {
|
||||||
|
|
|
||||||
|
|
@ -354,11 +354,13 @@ private[akka] class ClusterRouterPoolActor(
|
||||||
None
|
None
|
||||||
} else {
|
} else {
|
||||||
// find the node with least routees
|
// find the node with least routees
|
||||||
val numberOfRouteesPerNode: Map[Address, Int] =
|
val numberOfRouteesPerNode: Map[Address, Int] = {
|
||||||
currentRoutees.foldLeft(currentNodes.map(_ → 0).toMap.withDefaultValue(0)) { (acc, x) ⇒
|
val nodeMap: Map[Address, Int] = currentNodes.map(_ → 0).toMap.withDefaultValue(0)
|
||||||
|
currentRoutees.foldLeft(nodeMap) { (acc, x) ⇒
|
||||||
val address = fullAddress(x)
|
val address = fullAddress(x)
|
||||||
acc + (address → (acc(address) + 1))
|
acc + (address → (acc(address) + 1))
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
val (address, count) = numberOfRouteesPerNode.minBy(_._2)
|
val (address, count) = numberOfRouteesPerNode.minBy(_._2)
|
||||||
if (count < settings.maxInstancesPerNode) Some(address) else None
|
if (count < settings.maxInstancesPerNode) Some(address) else None
|
||||||
|
|
|
||||||
|
|
@ -10,6 +10,7 @@ import akka.remote.testkit.MultiNodeConfig
|
||||||
import akka.remote.testkit.MultiNodeSpec
|
import akka.remote.testkit.MultiNodeSpec
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
import akka.cluster.MemberStatus._
|
import akka.cluster.MemberStatus._
|
||||||
|
import akka.util.ccompat.imm._
|
||||||
|
|
||||||
object MinMembersBeforeUpMultiJvmSpec extends MultiNodeConfig {
|
object MinMembersBeforeUpMultiJvmSpec extends MultiNodeConfig {
|
||||||
val first = role("first")
|
val first = role("first")
|
||||||
|
|
@ -131,12 +132,12 @@ abstract class MinMembersBeforeUpBase(multiNodeConfig: MultiNodeConfig)
|
||||||
clusterView.refreshCurrentState()
|
clusterView.refreshCurrentState()
|
||||||
clusterView.members.map(_.address) should ===(expectedAddresses)
|
clusterView.members.map(_.address) should ===(expectedAddresses)
|
||||||
}
|
}
|
||||||
clusterView.members.map(_.status) should ===(Set(Joining))
|
clusterView.members.unsorted.map(_.status) should ===(Set(Joining))
|
||||||
// and it should not change
|
// and it should not change
|
||||||
1 to 5 foreach { _ ⇒
|
1 to 5 foreach { _ ⇒
|
||||||
Thread.sleep(1000)
|
Thread.sleep(1000)
|
||||||
clusterView.members.map(_.address) should ===(expectedAddresses)
|
clusterView.members.map(_.address) should ===(expectedAddresses)
|
||||||
clusterView.members.map(_.status) should ===(Set(Joining))
|
clusterView.members.unsorted.map(_.status) should ===(Set(Joining))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
enterBarrier("second-joined")
|
enterBarrier("second-joined")
|
||||||
|
|
|
||||||
|
|
@ -14,6 +14,7 @@ import com.typesafe.config.ConfigFactory
|
||||||
import scala.collection.immutable
|
import scala.collection.immutable
|
||||||
import scala.collection.immutable.SortedSet
|
import scala.collection.immutable.SortedSet
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
import akka.util.ccompat.imm._
|
||||||
|
|
||||||
object MultiDcHeartbeatTakingOverSpecMultiJvmSpec extends MultiNodeConfig {
|
object MultiDcHeartbeatTakingOverSpecMultiJvmSpec extends MultiNodeConfig {
|
||||||
val first = role("first") // alpha
|
val first = role("first") // alpha
|
||||||
|
|
@ -77,10 +78,10 @@ abstract class MultiDcHeartbeatTakingOverSpec extends MultiNodeSpec(MultiDcHeart
|
||||||
|
|
||||||
// these will be filled in during the initial phase of the test -----------
|
// these will be filled in during the initial phase of the test -----------
|
||||||
var expectedAlphaHeartbeaterNodes: SortedSet[Member] = SortedSet.empty
|
var expectedAlphaHeartbeaterNodes: SortedSet[Member] = SortedSet.empty
|
||||||
var expectedAlphaHeartbeaterRoles: SortedSet[RoleName] = SortedSet.empty
|
var expectedAlphaHeartbeaterRoles: List[RoleName] = List.empty
|
||||||
|
|
||||||
var expectedBetaHeartbeaterNodes: SortedSet[Member] = SortedSet.empty
|
var expectedBetaHeartbeaterNodes: SortedSet[Member] = SortedSet.empty
|
||||||
var expectedBetaHeartbeaterRoles: SortedSet[RoleName] = SortedSet.empty
|
var expectedBetaHeartbeaterRoles: List[RoleName] = List.empty
|
||||||
|
|
||||||
var expectedNoActiveHeartbeatSenderRoles: Set[RoleName] = Set.empty
|
var expectedNoActiveHeartbeatSenderRoles: Set[RoleName] = Set.empty
|
||||||
// end of these will be filled in during the initial phase of the test -----------
|
// end of these will be filled in during the initial phase of the test -----------
|
||||||
|
|
@ -191,8 +192,8 @@ abstract class MultiDcHeartbeatTakingOverSpec extends MultiNodeSpec(MultiDcHeart
|
||||||
private[cluster] def takeNOldestMembers(dataCenter: ClusterSettings.DataCenter, n: Int): immutable.SortedSet[Member] =
|
private[cluster] def takeNOldestMembers(dataCenter: ClusterSettings.DataCenter, n: Int): immutable.SortedSet[Member] =
|
||||||
membersByAge(dataCenter).take(n)
|
membersByAge(dataCenter).take(n)
|
||||||
|
|
||||||
private def membersAsRoles(ms: SortedSet[Member]): SortedSet[RoleName] = {
|
private def membersAsRoles(ms: SortedSet[Member]): List[RoleName] = {
|
||||||
val res = ms.flatMap(m ⇒ roleName(m.address))
|
val res = ms.toList.flatMap(m ⇒ roleName(m.address))
|
||||||
require(res.size == ms.size, s"Not all members were converted to roles! Got: ${ms}, found ${res}")
|
require(res.size == ms.size, s"Not all members were converted to roles! Got: ${ms}, found ${res}")
|
||||||
res
|
res
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -17,6 +17,7 @@ import akka.testkit._
|
||||||
import akka.testkit.TestEvent._
|
import akka.testkit.TestEvent._
|
||||||
import akka.actor.{ Actor, ActorRef, ActorSystem, Address, Deploy, PoisonPill, Props, RootActorPath }
|
import akka.actor.{ Actor, ActorRef, ActorSystem, Address, Deploy, PoisonPill, Props, RootActorPath }
|
||||||
import akka.event.Logging.ErrorLevel
|
import akka.event.Logging.ErrorLevel
|
||||||
|
import akka.util.ccompat._
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import scala.collection.immutable
|
import scala.collection.immutable
|
||||||
|
|
@ -24,6 +25,7 @@ import java.util.concurrent.ConcurrentHashMap
|
||||||
|
|
||||||
import akka.remote.DefaultFailureDetectorRegistry
|
import akka.remote.DefaultFailureDetectorRegistry
|
||||||
import akka.cluster.ClusterEvent.{ MemberEvent, MemberRemoved }
|
import akka.cluster.ClusterEvent.{ MemberEvent, MemberRemoved }
|
||||||
|
import akka.util.ccompat.imm._
|
||||||
|
|
||||||
import scala.concurrent.Await
|
import scala.concurrent.Await
|
||||||
|
|
||||||
|
|
@ -273,7 +275,7 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec with WatchedByCoro
|
||||||
* be determined from the `RoleName`.
|
* be determined from the `RoleName`.
|
||||||
*/
|
*/
|
||||||
def assertLeader(nodesInCluster: RoleName*): Unit =
|
def assertLeader(nodesInCluster: RoleName*): Unit =
|
||||||
if (nodesInCluster.contains(myself)) assertLeaderIn(nodesInCluster.to[immutable.Seq])
|
if (nodesInCluster.contains(myself)) assertLeaderIn(nodesInCluster.to(immutable.Seq))
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Assert that the cluster has elected the correct leader
|
* Assert that the cluster has elected the correct leader
|
||||||
|
|
@ -310,7 +312,7 @@ trait MultiNodeClusterSpec extends Suite with STMultiNodeSpec with WatchedByCoro
|
||||||
if (!canNotBePartOfMemberRing.isEmpty) // don't run this on an empty set
|
if (!canNotBePartOfMemberRing.isEmpty) // don't run this on an empty set
|
||||||
awaitAssert(canNotBePartOfMemberRing foreach (a ⇒ clusterView.members.map(_.address) should not contain (a)))
|
awaitAssert(canNotBePartOfMemberRing foreach (a ⇒ clusterView.members.map(_.address) should not contain (a)))
|
||||||
awaitAssert(clusterView.members.size should ===(numberOfMembers))
|
awaitAssert(clusterView.members.size should ===(numberOfMembers))
|
||||||
awaitAssert(clusterView.members.map(_.status) should ===(Set(MemberStatus.Up)))
|
awaitAssert(clusterView.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up)))
|
||||||
// clusterView.leader is updated by LeaderChanged, await that to be updated also
|
// clusterView.leader is updated by LeaderChanged, await that to be updated also
|
||||||
val expectedLeader = clusterView.members.collectFirst {
|
val expectedLeader = clusterView.members.collectFirst {
|
||||||
case m if m.dataCenter == cluster.settings.SelfDataCenter ⇒ m.address
|
case m if m.dataCenter == cluster.settings.SelfDataCenter ⇒ m.address
|
||||||
|
|
|
||||||
|
|
@ -7,6 +7,7 @@ package akka.cluster
|
||||||
import akka.remote.testkit.MultiNodeConfig
|
import akka.remote.testkit.MultiNodeConfig
|
||||||
import akka.remote.testkit.MultiNodeSpec
|
import akka.remote.testkit.MultiNodeSpec
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
|
import akka.util.ccompat.imm._
|
||||||
|
|
||||||
object NodeMembershipMultiJvmSpec extends MultiNodeConfig {
|
object NodeMembershipMultiJvmSpec extends MultiNodeConfig {
|
||||||
val first = role("first")
|
val first = role("first")
|
||||||
|
|
@ -40,7 +41,7 @@ abstract class NodeMembershipSpec
|
||||||
cluster.join(first)
|
cluster.join(first)
|
||||||
awaitAssert(clusterView.members.size should ===(2))
|
awaitAssert(clusterView.members.size should ===(2))
|
||||||
assertMembers(clusterView.members, first, second)
|
assertMembers(clusterView.members, first, second)
|
||||||
awaitAssert(clusterView.members.map(_.status) should ===(Set(MemberStatus.Up)))
|
awaitAssert(clusterView.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up)))
|
||||||
}
|
}
|
||||||
|
|
||||||
enterBarrier("after-1")
|
enterBarrier("after-1")
|
||||||
|
|
@ -54,7 +55,7 @@ abstract class NodeMembershipSpec
|
||||||
|
|
||||||
awaitAssert(clusterView.members.size should ===(3))
|
awaitAssert(clusterView.members.size should ===(3))
|
||||||
assertMembers(clusterView.members, first, second, third)
|
assertMembers(clusterView.members, first, second, third)
|
||||||
awaitAssert(clusterView.members.map(_.status) should ===(Set(MemberStatus.Up)))
|
awaitAssert(clusterView.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up)))
|
||||||
|
|
||||||
enterBarrier("after-2")
|
enterBarrier("after-2")
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -19,6 +19,7 @@ import akka.actor.Actor
|
||||||
import akka.actor.RootActorPath
|
import akka.actor.RootActorPath
|
||||||
import akka.cluster.MemberStatus._
|
import akka.cluster.MemberStatus._
|
||||||
import akka.actor.Deploy
|
import akka.actor.Deploy
|
||||||
|
import akka.util.ccompat.imm._
|
||||||
|
|
||||||
object RestartFirstSeedNodeMultiJvmSpec extends MultiNodeConfig {
|
object RestartFirstSeedNodeMultiJvmSpec extends MultiNodeConfig {
|
||||||
val seed1 = role("seed1")
|
val seed1 = role("seed1")
|
||||||
|
|
@ -97,7 +98,7 @@ abstract class RestartFirstSeedNodeSpec
|
||||||
runOn(seed1) {
|
runOn(seed1) {
|
||||||
Cluster(seed1System).joinSeedNodes(seedNodes)
|
Cluster(seed1System).joinSeedNodes(seedNodes)
|
||||||
awaitAssert(Cluster(seed1System).readView.members.size should ===(3))
|
awaitAssert(Cluster(seed1System).readView.members.size should ===(3))
|
||||||
awaitAssert(Cluster(seed1System).readView.members.map(_.status) should ===(Set(Up)))
|
awaitAssert(Cluster(seed1System).readView.members.unsorted.map(_.status) should ===(Set(Up)))
|
||||||
}
|
}
|
||||||
runOn(seed2, seed3) {
|
runOn(seed2, seed3) {
|
||||||
cluster.joinSeedNodes(seedNodes)
|
cluster.joinSeedNodes(seedNodes)
|
||||||
|
|
@ -116,7 +117,7 @@ abstract class RestartFirstSeedNodeSpec
|
||||||
Cluster(restartedSeed1System).joinSeedNodes(seedNodes)
|
Cluster(restartedSeed1System).joinSeedNodes(seedNodes)
|
||||||
within(20.seconds) {
|
within(20.seconds) {
|
||||||
awaitAssert(Cluster(restartedSeed1System).readView.members.size should ===(3))
|
awaitAssert(Cluster(restartedSeed1System).readView.members.size should ===(3))
|
||||||
awaitAssert(Cluster(restartedSeed1System).readView.members.map(_.status) should ===(Set(Up)))
|
awaitAssert(Cluster(restartedSeed1System).readView.members.unsorted.map(_.status) should ===(Set(Up)))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
runOn(seed2, seed3) {
|
runOn(seed2, seed3) {
|
||||||
|
|
|
||||||
|
|
@ -18,6 +18,7 @@ import akka.remote.testkit.MultiNodeConfig
|
||||||
import akka.remote.testkit.MultiNodeSpec
|
import akka.remote.testkit.MultiNodeSpec
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
|
import akka.util.ccompat.imm._
|
||||||
|
|
||||||
object RestartNode2SpecMultiJvmSpec extends MultiNodeConfig {
|
object RestartNode2SpecMultiJvmSpec extends MultiNodeConfig {
|
||||||
val seed1 = role("seed1")
|
val seed1 = role("seed1")
|
||||||
|
|
@ -99,7 +100,7 @@ abstract class RestartNode2SpecSpec
|
||||||
runOn(seed1) {
|
runOn(seed1) {
|
||||||
Cluster(seed1System).joinSeedNodes(seedNodes)
|
Cluster(seed1System).joinSeedNodes(seedNodes)
|
||||||
awaitAssert(Cluster(seed1System).readView.members.size should be(2))
|
awaitAssert(Cluster(seed1System).readView.members.size should be(2))
|
||||||
awaitAssert(Cluster(seed1System).readView.members.map(_.status) should be(Set(Up)))
|
awaitAssert(Cluster(seed1System).readView.members.unsorted.map(_.status) should be(Set(Up)))
|
||||||
}
|
}
|
||||||
runOn(seed2) {
|
runOn(seed2) {
|
||||||
cluster.joinSeedNodes(seedNodes)
|
cluster.joinSeedNodes(seedNodes)
|
||||||
|
|
@ -118,7 +119,7 @@ abstract class RestartNode2SpecSpec
|
||||||
Cluster(restartedSeed1System).joinSeedNodes(seedNodes)
|
Cluster(restartedSeed1System).joinSeedNodes(seedNodes)
|
||||||
within(30.seconds) {
|
within(30.seconds) {
|
||||||
awaitAssert(Cluster(restartedSeed1System).readView.members.size should be(2))
|
awaitAssert(Cluster(restartedSeed1System).readView.members.size should be(2))
|
||||||
awaitAssert(Cluster(restartedSeed1System).readView.members.map(_.status) should be(Set(Up)))
|
awaitAssert(Cluster(restartedSeed1System).readView.members.unsorted.map(_.status) should be(Set(Up)))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
runOn(seed2) {
|
runOn(seed2) {
|
||||||
|
|
|
||||||
|
|
@ -18,6 +18,7 @@ import akka.remote.testkit.MultiNodeSpec
|
||||||
import akka.remote.transport.ThrottlerTransportAdapter.Direction
|
import akka.remote.transport.ThrottlerTransportAdapter.Direction
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
|
import akka.util.ccompat.imm._
|
||||||
|
|
||||||
object RestartNode3MultiJvmSpec extends MultiNodeConfig {
|
object RestartNode3MultiJvmSpec extends MultiNodeConfig {
|
||||||
val first = role("first")
|
val first = role("first")
|
||||||
|
|
@ -133,7 +134,7 @@ abstract class RestartNode3Spec
|
||||||
runOn(second) {
|
runOn(second) {
|
||||||
Cluster(restartedSecondSystem).joinSeedNodes(seedNodes)
|
Cluster(restartedSecondSystem).joinSeedNodes(seedNodes)
|
||||||
awaitAssert(Cluster(restartedSecondSystem).readView.members.size should ===(3))
|
awaitAssert(Cluster(restartedSecondSystem).readView.members.size should ===(3))
|
||||||
awaitAssert(Cluster(restartedSecondSystem).readView.members.map(_.status) should ===(Set(Up)))
|
awaitAssert(Cluster(restartedSecondSystem).readView.members.unsorted.map(_.status) should ===(Set(Up)))
|
||||||
}
|
}
|
||||||
runOn(first, third) {
|
runOn(first, third) {
|
||||||
awaitAssert {
|
awaitAssert {
|
||||||
|
|
|
||||||
|
|
@ -23,6 +23,7 @@ import akka.remote.testkit.MultiNodeConfig
|
||||||
import akka.remote.testkit.MultiNodeSpec
|
import akka.remote.testkit.MultiNodeSpec
|
||||||
import akka.testkit._
|
import akka.testkit._
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
|
import akka.util.ccompat.imm._
|
||||||
|
|
||||||
object RestartNodeMultiJvmSpec extends MultiNodeConfig {
|
object RestartNodeMultiJvmSpec extends MultiNodeConfig {
|
||||||
val first = role("first")
|
val first = role("first")
|
||||||
|
|
@ -121,7 +122,7 @@ abstract class RestartNodeSpec
|
||||||
runOn(second) {
|
runOn(second) {
|
||||||
Cluster(secondSystem).joinSeedNodes(seedNodes)
|
Cluster(secondSystem).joinSeedNodes(seedNodes)
|
||||||
awaitAssert(Cluster(secondSystem).readView.members.size should ===(3))
|
awaitAssert(Cluster(secondSystem).readView.members.size should ===(3))
|
||||||
awaitAssert(Cluster(secondSystem).readView.members.map(_.status) should ===(Set(Up)))
|
awaitAssert(Cluster(secondSystem).readView.members.unsorted.map(_.status) should ===(Set(Up)))
|
||||||
}
|
}
|
||||||
enterBarrier("started")
|
enterBarrier("started")
|
||||||
|
|
||||||
|
|
@ -139,7 +140,7 @@ abstract class RestartNodeSpec
|
||||||
runOn(second) {
|
runOn(second) {
|
||||||
Cluster(restartedSecondSystem).joinSeedNodes(seedNodes)
|
Cluster(restartedSecondSystem).joinSeedNodes(seedNodes)
|
||||||
awaitAssert(Cluster(restartedSecondSystem).readView.members.size should ===(3))
|
awaitAssert(Cluster(restartedSecondSystem).readView.members.size should ===(3))
|
||||||
awaitAssert(Cluster(restartedSecondSystem).readView.members.map(_.status) should ===(Set(Up)))
|
awaitAssert(Cluster(restartedSecondSystem).readView.members.unsorted.map(_.status) should ===(Set(Up)))
|
||||||
}
|
}
|
||||||
runOn(first, third) {
|
runOn(first, third) {
|
||||||
awaitAssert {
|
awaitAssert {
|
||||||
|
|
|
||||||
|
|
@ -345,7 +345,7 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig {
|
||||||
class PhiObserver extends Actor with ActorLogging {
|
class PhiObserver extends Actor with ActorLogging {
|
||||||
val cluster = Cluster(context.system)
|
val cluster = Cluster(context.system)
|
||||||
var reportTo: Option[ActorRef] = None
|
var reportTo: Option[ActorRef] = None
|
||||||
val emptyPhiByNode = Map.empty[Address, PhiValue].withDefault(address ⇒ PhiValue(address, 0, 0, 0.0))
|
val emptyPhiByNode: Map[Address, PhiValue] = Map.empty[Address, PhiValue].withDefault(address ⇒ PhiValue(address, 0, 0, 0.0))
|
||||||
var phiByNode = emptyPhiByNode
|
var phiByNode = emptyPhiByNode
|
||||||
var nodes = Set.empty[Address]
|
var nodes = Set.empty[Address]
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -19,6 +19,7 @@ import akka.testkit._
|
||||||
import akka.actor.Props
|
import akka.actor.Props
|
||||||
import akka.cluster.MultiNodeClusterSpec.EndActor
|
import akka.cluster.MultiNodeClusterSpec.EndActor
|
||||||
import akka.remote.RARP
|
import akka.remote.RARP
|
||||||
|
import akka.util.ccompat.imm._
|
||||||
|
|
||||||
object UnreachableNodeJoinsAgainMultiNodeConfig extends MultiNodeConfig {
|
object UnreachableNodeJoinsAgainMultiNodeConfig extends MultiNodeConfig {
|
||||||
val first = role("first")
|
val first = role("first")
|
||||||
|
|
@ -185,7 +186,7 @@ abstract class UnreachableNodeJoinsAgainSpec
|
||||||
within(30 seconds) {
|
within(30 seconds) {
|
||||||
awaitAssert(Cluster(freshSystem).readView.members.map(_.address) should contain(victimAddress))
|
awaitAssert(Cluster(freshSystem).readView.members.map(_.address) should contain(victimAddress))
|
||||||
awaitAssert(Cluster(freshSystem).readView.members.size should ===(expectedNumberOfMembers))
|
awaitAssert(Cluster(freshSystem).readView.members.size should ===(expectedNumberOfMembers))
|
||||||
awaitAssert(Cluster(freshSystem).readView.members.map(_.status) should ===(Set(MemberStatus.Up)))
|
awaitAssert(Cluster(freshSystem).readView.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up)))
|
||||||
}
|
}
|
||||||
|
|
||||||
// signal to master node that victim is done
|
// signal to master node that victim is done
|
||||||
|
|
|
||||||
|
|
@ -37,7 +37,7 @@ class ClusterDomainEventSpec extends WordSpec with Matchers {
|
||||||
val selfDummyAddress = UniqueAddress(Address("akka.tcp", "sys", "selfDummy", 2552), 17L)
|
val selfDummyAddress = UniqueAddress(Address("akka.tcp", "sys", "selfDummy", 2552), 17L)
|
||||||
|
|
||||||
private[cluster] def converge(gossip: Gossip): (Gossip, Set[UniqueAddress]) =
|
private[cluster] def converge(gossip: Gossip): (Gossip, Set[UniqueAddress]) =
|
||||||
((gossip, Set.empty[UniqueAddress]) /: gossip.members) { case ((gs, as), m) ⇒ (gs.seen(m.uniqueAddress), as + m.uniqueAddress) }
|
gossip.members.foldLeft((gossip, Set.empty[UniqueAddress])) { case ((gs, as), m) ⇒ (gs.seen(m.uniqueAddress), as + m.uniqueAddress) }
|
||||||
|
|
||||||
private def state(g: Gossip): MembershipState =
|
private def state(g: Gossip): MembershipState =
|
||||||
state(g, selfDummyAddress)
|
state(g, selfDummyAddress)
|
||||||
|
|
|
||||||
|
|
@ -18,7 +18,7 @@ class ReachabilityPerfSpec extends WordSpec with Matchers {
|
||||||
val node = Address("akka.tcp", "sys", "a", 2552)
|
val node = Address("akka.tcp", "sys", "a", 2552)
|
||||||
|
|
||||||
private def createReachabilityOfSize(base: Reachability, size: Int): Reachability =
|
private def createReachabilityOfSize(base: Reachability, size: Int): Reachability =
|
||||||
(base /: (1 to size)) {
|
(1 to size).foldLeft(base) {
|
||||||
case (r, i) ⇒
|
case (r, i) ⇒
|
||||||
val observer = UniqueAddress(address.copy(host = Some("node-" + i)), i.toLong)
|
val observer = UniqueAddress(address.copy(host = Some("node-" + i)), i.toLong)
|
||||||
val j = if (i == size) 1 else i + 1
|
val j = if (i == size) 1 else i + 1
|
||||||
|
|
@ -29,9 +29,9 @@ class ReachabilityPerfSpec extends WordSpec with Matchers {
|
||||||
private def addUnreachable(base: Reachability, count: Int): Reachability = {
|
private def addUnreachable(base: Reachability, count: Int): Reachability = {
|
||||||
val observers = base.allObservers.take(count)
|
val observers = base.allObservers.take(count)
|
||||||
val subjects = Stream.continually(base.allObservers).flatten.iterator
|
val subjects = Stream.continually(base.allObservers).flatten.iterator
|
||||||
(base /: observers) {
|
observers.foldLeft(base) {
|
||||||
case (r, o) ⇒
|
case (r, o) ⇒
|
||||||
(r /: (1 to 5)) { case (r, _) ⇒ r.unreachable(o, subjects.next()) }
|
(1 to 5).foldLeft(r) { case (r, _) ⇒ r.unreachable(o, subjects.next()) }
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -12,14 +12,14 @@ object VectorClockPerfSpec {
|
||||||
import VectorClock._
|
import VectorClock._
|
||||||
|
|
||||||
def createVectorClockOfSize(size: Int): (VectorClock, SortedSet[Node]) =
|
def createVectorClockOfSize(size: Int): (VectorClock, SortedSet[Node]) =
|
||||||
((VectorClock(), SortedSet.empty[Node]) /: (1 to size)) {
|
(1 to size).foldLeft((VectorClock(), SortedSet.empty[Node])) {
|
||||||
case ((vc, nodes), i) ⇒
|
case ((vc, nodes), i) ⇒
|
||||||
val node = Node(i.toString)
|
val node = Node(i.toString)
|
||||||
(vc :+ node, nodes + node)
|
(vc :+ node, nodes + node)
|
||||||
}
|
}
|
||||||
|
|
||||||
def copyVectorClock(vc: VectorClock): VectorClock = {
|
def copyVectorClock(vc: VectorClock): VectorClock = {
|
||||||
val versions = (TreeMap.empty[Node, Long] /: vc.versions) {
|
val versions = vc.versions.foldLeft(TreeMap.empty[Node, Long]) {
|
||||||
case (versions, (n, t)) ⇒ versions.updated(Node.fromHash(n), t)
|
case (versions, (n, t)) ⇒ versions.updated(Node.fromHash(n), t)
|
||||||
}
|
}
|
||||||
vc.copy(versions = versions)
|
vc.copy(versions = versions)
|
||||||
|
|
|
||||||
|
|
@ -98,9 +98,7 @@ trait JavaLoggingAdapter extends LoggingAdapter {
|
||||||
|
|
||||||
if (loggingExecutionContext.isDefined) {
|
if (loggingExecutionContext.isDefined) {
|
||||||
implicit val context = loggingExecutionContext.get
|
implicit val context = loggingExecutionContext.get
|
||||||
Future(logger.log(record)).onFailure {
|
Future(logger.log(record)).failed.foreach { _.printStackTrace() }
|
||||||
case thrown: Throwable ⇒ thrown.printStackTrace()
|
|
||||||
}
|
|
||||||
} else
|
} else
|
||||||
logger.log(record)
|
logger.log(record)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -11,6 +11,7 @@ import akka.annotation.InternalApi
|
||||||
import akka.cluster.ddata.Key.KeyId
|
import akka.cluster.ddata.Key.KeyId
|
||||||
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation
|
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation
|
||||||
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation.NoDeltaPlaceholder
|
import akka.cluster.ddata.Replicator.Internal.DeltaPropagation.NoDeltaPlaceholder
|
||||||
|
import akka.util.ccompat._
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API: Used by the Replicator actor.
|
* INTERNAL API: Used by the Replicator actor.
|
||||||
|
|
@ -144,7 +145,7 @@ import akka.cluster.ddata.Replicator.Internal.DeltaPropagation.NoDeltaPlaceholde
|
||||||
}
|
}
|
||||||
|
|
||||||
private def deltaEntriesAfter(entries: TreeMap[Long, ReplicatedData], version: Long): TreeMap[Long, ReplicatedData] =
|
private def deltaEntriesAfter(entries: TreeMap[Long, ReplicatedData], version: Long): TreeMap[Long, ReplicatedData] =
|
||||||
entries.from(version) match {
|
entries.rangeFrom(version) match {
|
||||||
case ntrs if ntrs.isEmpty ⇒ ntrs
|
case ntrs if ntrs.isEmpty ⇒ ntrs
|
||||||
case ntrs if ntrs.firstKey == version ⇒ ntrs.tail // exclude first, i.e. version j that was already sent
|
case ntrs if ntrs.firstKey == version ⇒ ntrs.tail // exclude first, i.e. version j that was already sent
|
||||||
case ntrs ⇒ ntrs
|
case ntrs ⇒ ntrs
|
||||||
|
|
|
||||||
|
|
@ -53,6 +53,7 @@ import scala.collection.immutable.TreeSet
|
||||||
import akka.cluster.MemberStatus
|
import akka.cluster.MemberStatus
|
||||||
import scala.annotation.varargs
|
import scala.annotation.varargs
|
||||||
import akka.util.JavaDurationConverters._
|
import akka.util.JavaDurationConverters._
|
||||||
|
import akka.util.ccompat._
|
||||||
|
|
||||||
object ReplicatorSettings {
|
object ReplicatorSettings {
|
||||||
|
|
||||||
|
|
@ -1109,13 +1110,13 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
override def createDeltaPropagation(deltas: Map[KeyId, (ReplicatedData, Long, Long)]): DeltaPropagation = {
|
override def createDeltaPropagation(deltas: Map[KeyId, (ReplicatedData, Long, Long)]): DeltaPropagation = {
|
||||||
// Important to include the pruning state in the deltas. For example if the delta is based
|
// Important to include the pruning state in the deltas. For example if the delta is based
|
||||||
// on an entry that has been pruned but that has not yet been performed on the target node.
|
// on an entry that has been pruned but that has not yet been performed on the target node.
|
||||||
DeltaPropagation(selfUniqueAddress, reply = false, deltas.collect {
|
DeltaPropagation(selfUniqueAddress, reply = false, deltas.iterator.collect {
|
||||||
case (key, (d, fromSeqNr, toSeqNr)) if d != NoDeltaPlaceholder ⇒
|
case (key, (d, fromSeqNr, toSeqNr)) if d != NoDeltaPlaceholder ⇒
|
||||||
getData(key) match {
|
getData(key) match {
|
||||||
case Some(envelope) ⇒ key → Delta(envelope.copy(data = d), fromSeqNr, toSeqNr)
|
case Some(envelope) ⇒ key → Delta(envelope.copy(data = d), fromSeqNr, toSeqNr)
|
||||||
case None ⇒ key → Delta(DataEnvelope(d), fromSeqNr, toSeqNr)
|
case None ⇒ key → Delta(DataEnvelope(d), fromSeqNr, toSeqNr)
|
||||||
}
|
}
|
||||||
}(collection.breakOut))
|
}.toMap)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
val deltaPropagationTask: Option[Cancellable] =
|
val deltaPropagationTask: Option[Cancellable] =
|
||||||
|
|
@ -1461,9 +1462,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
}
|
}
|
||||||
|
|
||||||
def receiveGetKeyIds(): Unit = {
|
def receiveGetKeyIds(): Unit = {
|
||||||
val keys: Set[KeyId] = dataEntries.collect {
|
val keys: Set[KeyId] = dataEntries.iterator.collect {
|
||||||
case (key, (DataEnvelope(data, _, _), _)) if data != DeletedData ⇒ key
|
case (key, (DataEnvelope(data, _, _), _)) if data != DeletedData ⇒ key
|
||||||
}(collection.breakOut)
|
}.to(immutable.Set)
|
||||||
replyTo ! GetKeyIdsResult(keys)
|
replyTo ! GetKeyIdsResult(keys)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -1700,14 +1701,14 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
if (keys.nonEmpty) {
|
if (keys.nonEmpty) {
|
||||||
if (log.isDebugEnabled)
|
if (log.isDebugEnabled)
|
||||||
log.debug("Sending gossip to [{}], containing [{}]", replyTo.path.address, keys.mkString(", "))
|
log.debug("Sending gossip to [{}], containing [{}]", replyTo.path.address, keys.mkString(", "))
|
||||||
val g = Gossip(keys.map(k ⇒ k → getData(k).get)(collection.breakOut), sendBack = otherDifferentKeys.nonEmpty)
|
val g = Gossip(keys.iterator.map(k ⇒ k → getData(k).get).toMap, sendBack = otherDifferentKeys.nonEmpty)
|
||||||
replyTo ! g
|
replyTo ! g
|
||||||
}
|
}
|
||||||
val myMissingKeys = otherKeys diff myKeys
|
val myMissingKeys = otherKeys diff myKeys
|
||||||
if (myMissingKeys.nonEmpty) {
|
if (myMissingKeys.nonEmpty) {
|
||||||
if (log.isDebugEnabled)
|
if (log.isDebugEnabled)
|
||||||
log.debug("Sending gossip status to [{}], requesting missing [{}]", replyTo.path.address, myMissingKeys.mkString(", "))
|
log.debug("Sending gossip status to [{}], requesting missing [{}]", replyTo.path.address, myMissingKeys.mkString(", "))
|
||||||
val status = Status(myMissingKeys.map(k ⇒ k → NotFoundDigest)(collection.breakOut), chunk, totChunks)
|
val status = Status(myMissingKeys.iterator.map(k ⇒ k → NotFoundDigest).toMap, chunk, totChunks)
|
||||||
replyTo ! status
|
replyTo ! status
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -1846,9 +1847,9 @@ final class Replicator(settings: ReplicatorSettings) extends Actor with ActorLog
|
||||||
|
|
||||||
def initRemovedNodePruning(): Unit = {
|
def initRemovedNodePruning(): Unit = {
|
||||||
// initiate pruning for removed nodes
|
// initiate pruning for removed nodes
|
||||||
val removedSet: Set[UniqueAddress] = removedNodes.collect {
|
val removedSet: Set[UniqueAddress] = removedNodes.iterator.collect {
|
||||||
case (r, t) if ((allReachableClockTime - t) > maxPruningDisseminationNanos) ⇒ r
|
case (r, t) if ((allReachableClockTime - t) > maxPruningDisseminationNanos) ⇒ r
|
||||||
}(collection.breakOut)
|
}.to(immutable.Set)
|
||||||
|
|
||||||
if (removedSet.nonEmpty) {
|
if (removedSet.nonEmpty) {
|
||||||
for ((key, (envelope, _)) ← dataEntries; removed ← removedSet) {
|
for ((key, (envelope, _)) ← dataEntries; removed ← removedSet) {
|
||||||
|
|
|
||||||
|
|
@ -12,7 +12,7 @@ import java.util.TreeSet
|
||||||
|
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
import scala.collection.JavaConverters._
|
import scala.collection.JavaConverters._
|
||||||
import scala.collection.breakOut
|
import scala.collection.immutable
|
||||||
|
|
||||||
import akka.actor.ExtendedActorSystem
|
import akka.actor.ExtendedActorSystem
|
||||||
import akka.cluster.ddata._
|
import akka.cluster.ddata._
|
||||||
|
|
@ -28,6 +28,7 @@ import java.io.NotSerializableException
|
||||||
import akka.actor.ActorRef
|
import akka.actor.ActorRef
|
||||||
import akka.cluster.ddata.protobuf.msg.ReplicatorMessages.OtherMessage
|
import akka.cluster.ddata.protobuf.msg.ReplicatorMessages.OtherMessage
|
||||||
import akka.serialization.Serialization
|
import akka.serialization.Serialization
|
||||||
|
import akka.util.ccompat._
|
||||||
|
|
||||||
private object ReplicatedDataSerializer {
|
private object ReplicatedDataSerializer {
|
||||||
/*
|
/*
|
||||||
|
|
@ -472,7 +473,7 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
||||||
private def orsetDeltaGroupFromBinary(bytes: Array[Byte]): ORSet.DeltaGroup[Any] = {
|
private def orsetDeltaGroupFromBinary(bytes: Array[Byte]): ORSet.DeltaGroup[Any] = {
|
||||||
val deltaGroup = rd.ORSetDeltaGroup.parseFrom(bytes)
|
val deltaGroup = rd.ORSetDeltaGroup.parseFrom(bytes)
|
||||||
val ops: Vector[ORSet.DeltaOp] =
|
val ops: Vector[ORSet.DeltaOp] =
|
||||||
deltaGroup.getEntriesList.asScala.map { entry ⇒
|
deltaGroup.getEntriesList.asScala.iterator.map { entry ⇒
|
||||||
if (entry.getOperation == rd.ORSetDeltaOp.Add)
|
if (entry.getOperation == rd.ORSetDeltaOp.Add)
|
||||||
ORSet.AddDeltaOp(orsetFromProto(entry.getUnderlying))
|
ORSet.AddDeltaOp(orsetFromProto(entry.getUnderlying))
|
||||||
else if (entry.getOperation == rd.ORSetDeltaOp.Remove)
|
else if (entry.getOperation == rd.ORSetDeltaOp.Remove)
|
||||||
|
|
@ -481,7 +482,7 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
||||||
ORSet.FullStateDeltaOp(orsetFromProto(entry.getUnderlying))
|
ORSet.FullStateDeltaOp(orsetFromProto(entry.getUnderlying))
|
||||||
else
|
else
|
||||||
throw new NotSerializableException(s"Unknow ORSet delta operation ${entry.getOperation}")
|
throw new NotSerializableException(s"Unknow ORSet delta operation ${entry.getOperation}")
|
||||||
}(collection.breakOut)
|
}.to(immutable.Vector)
|
||||||
ORSet.DeltaGroup(ops)
|
ORSet.DeltaGroup(ops)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -538,8 +539,8 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
||||||
gcounterFromProto(rd.GCounter.parseFrom(bytes))
|
gcounterFromProto(rd.GCounter.parseFrom(bytes))
|
||||||
|
|
||||||
def gcounterFromProto(gcounter: rd.GCounter): GCounter = {
|
def gcounterFromProto(gcounter: rd.GCounter): GCounter = {
|
||||||
new GCounter(state = gcounter.getEntriesList.asScala.map(entry ⇒
|
new GCounter(state = gcounter.getEntriesList.asScala.iterator.map(entry ⇒
|
||||||
uniqueAddressFromProto(entry.getNode) → BigInt(entry.getValue.toByteArray))(breakOut))
|
uniqueAddressFromProto(entry.getNode) → BigInt(entry.getValue.toByteArray)).toMap)
|
||||||
}
|
}
|
||||||
|
|
||||||
def pncounterToProto(pncounter: PNCounter): rd.PNCounter =
|
def pncounterToProto(pncounter: PNCounter): rd.PNCounter =
|
||||||
|
|
@ -670,7 +671,7 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
||||||
private def ormapDeltaGroupOpsFromBinary(bytes: Array[Byte]): scala.collection.immutable.IndexedSeq[ORMap.DeltaOp] = {
|
private def ormapDeltaGroupOpsFromBinary(bytes: Array[Byte]): scala.collection.immutable.IndexedSeq[ORMap.DeltaOp] = {
|
||||||
val deltaGroup = rd.ORMapDeltaGroup.parseFrom(bytes)
|
val deltaGroup = rd.ORMapDeltaGroup.parseFrom(bytes)
|
||||||
val ops: Vector[ORMap.DeltaOp] =
|
val ops: Vector[ORMap.DeltaOp] =
|
||||||
deltaGroup.getEntriesList.asScala.map { entry ⇒
|
deltaGroup.getEntriesList.asScala.iterator.map { entry ⇒
|
||||||
if (entry.getOperation == rd.ORMapDeltaOp.ORMapPut) {
|
if (entry.getOperation == rd.ORMapDeltaOp.ORMapPut) {
|
||||||
val map = singleMapEntryFromProto(entry.getEntryDataList, (v: dm.OtherMessage) ⇒ otherMessageFromProto(v).asInstanceOf[ReplicatedData])
|
val map = singleMapEntryFromProto(entry.getEntryDataList, (v: dm.OtherMessage) ⇒ otherMessageFromProto(v).asInstanceOf[ReplicatedData])
|
||||||
ORMap.PutDeltaOp(ORSet.AddDeltaOp(orsetFromProto(entry.getUnderlying)), map.head, zeroTagFromCode(entry.getZeroTag))
|
ORMap.PutDeltaOp(ORSet.AddDeltaOp(orsetFromProto(entry.getUnderlying)), map.head, zeroTagFromCode(entry.getZeroTag))
|
||||||
|
|
@ -684,31 +685,31 @@ class ReplicatedDataSerializer(val system: ExtendedActorSystem)
|
||||||
ORMap.UpdateDeltaOp(ORSet.AddDeltaOp(orsetFromProto(entry.getUnderlying)), map, zeroTagFromCode(entry.getZeroTag))
|
ORMap.UpdateDeltaOp(ORSet.AddDeltaOp(orsetFromProto(entry.getUnderlying)), map, zeroTagFromCode(entry.getZeroTag))
|
||||||
} else
|
} else
|
||||||
throw new NotSerializableException(s"Unknown ORMap delta operation ${entry.getOperation}")
|
throw new NotSerializableException(s"Unknown ORMap delta operation ${entry.getOperation}")
|
||||||
}(collection.breakOut)
|
}.to(immutable.Vector)
|
||||||
ops
|
ops
|
||||||
}
|
}
|
||||||
|
|
||||||
private def ormapPutToProto(deltaOp: ORMap.PutDeltaOp[_, _]): rd.ORMapDeltaGroup = {
|
private def ormapPutToProto(deltaOp: ORMap.PutDeltaOp[_, _]): rd.ORMapDeltaGroup = {
|
||||||
ormapDeltaGroupOpsToProto(scala.collection.immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
|
ormapDeltaGroupOpsToProto(immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
|
||||||
}
|
}
|
||||||
|
|
||||||
private def ormapRemoveToProto(deltaOp: ORMap.RemoveDeltaOp[_, _]): rd.ORMapDeltaGroup = {
|
private def ormapRemoveToProto(deltaOp: ORMap.RemoveDeltaOp[_, _]): rd.ORMapDeltaGroup = {
|
||||||
ormapDeltaGroupOpsToProto(scala.collection.immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
|
ormapDeltaGroupOpsToProto(immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
|
||||||
}
|
}
|
||||||
|
|
||||||
private def ormapRemoveKeyToProto(deltaOp: ORMap.RemoveKeyDeltaOp[_, _]): rd.ORMapDeltaGroup = {
|
private def ormapRemoveKeyToProto(deltaOp: ORMap.RemoveKeyDeltaOp[_, _]): rd.ORMapDeltaGroup = {
|
||||||
ormapDeltaGroupOpsToProto(scala.collection.immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
|
ormapDeltaGroupOpsToProto(immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
|
||||||
}
|
}
|
||||||
|
|
||||||
private def ormapUpdateToProto(deltaOp: ORMap.UpdateDeltaOp[_, _]): rd.ORMapDeltaGroup = {
|
private def ormapUpdateToProto(deltaOp: ORMap.UpdateDeltaOp[_, _]): rd.ORMapDeltaGroup = {
|
||||||
ormapDeltaGroupOpsToProto(scala.collection.immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
|
ormapDeltaGroupOpsToProto(immutable.IndexedSeq(deltaOp.asInstanceOf[ORMap.DeltaOp]))
|
||||||
}
|
}
|
||||||
|
|
||||||
private def ormapDeltaGroupToProto(deltaGroup: ORMap.DeltaGroup[_, _]): rd.ORMapDeltaGroup = {
|
private def ormapDeltaGroupToProto(deltaGroup: ORMap.DeltaGroup[_, _]): rd.ORMapDeltaGroup = {
|
||||||
ormapDeltaGroupOpsToProto(deltaGroup.ops)
|
ormapDeltaGroupOpsToProto(deltaGroup.ops)
|
||||||
}
|
}
|
||||||
|
|
||||||
private def ormapDeltaGroupOpsToProto(deltaGroupOps: scala.collection.immutable.IndexedSeq[ORMap.DeltaOp]): rd.ORMapDeltaGroup = {
|
private def ormapDeltaGroupOpsToProto(deltaGroupOps: immutable.IndexedSeq[ORMap.DeltaOp]): rd.ORMapDeltaGroup = {
|
||||||
def createEntry(opType: rd.ORMapDeltaOp, u: ORSet[_], m: Map[_, _], zt: Int) = {
|
def createEntry(opType: rd.ORMapDeltaOp, u: ORSet[_], m: Map[_, _], zt: Int) = {
|
||||||
if (m.size > 1 && opType != rd.ORMapDeltaOp.ORMapUpdate)
|
if (m.size > 1 && opType != rd.ORMapDeltaOp.ORMapUpdate)
|
||||||
throw new IllegalArgumentException("Invalid size of ORMap delta map")
|
throw new IllegalArgumentException("Invalid size of ORMap delta map")
|
||||||
|
|
|
||||||
|
|
@ -7,7 +7,7 @@ package akka.cluster.ddata.protobuf
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import java.util.concurrent.TimeUnit
|
import java.util.concurrent.TimeUnit
|
||||||
import scala.collection.JavaConverters._
|
import scala.collection.JavaConverters._
|
||||||
import scala.collection.breakOut
|
import scala.collection.immutable
|
||||||
import scala.concurrent.duration.Duration
|
import scala.concurrent.duration.Duration
|
||||||
import akka.actor.ExtendedActorSystem
|
import akka.actor.ExtendedActorSystem
|
||||||
import akka.cluster.Member
|
import akka.cluster.Member
|
||||||
|
|
@ -32,6 +32,7 @@ import akka.actor.Address
|
||||||
import akka.cluster.ddata.VersionVector
|
import akka.cluster.ddata.VersionVector
|
||||||
import akka.annotation.InternalApi
|
import akka.annotation.InternalApi
|
||||||
import akka.cluster.ddata.PruningState.PruningPerformed
|
import akka.cluster.ddata.PruningState.PruningPerformed
|
||||||
|
import akka.util.ccompat._
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
@ -268,8 +269,8 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
||||||
private def statusFromBinary(bytes: Array[Byte]): Status = {
|
private def statusFromBinary(bytes: Array[Byte]): Status = {
|
||||||
val status = dm.Status.parseFrom(bytes)
|
val status = dm.Status.parseFrom(bytes)
|
||||||
Status(
|
Status(
|
||||||
status.getEntriesList.asScala.map(e ⇒
|
status.getEntriesList.asScala.iterator.map(e ⇒
|
||||||
e.getKey → AkkaByteString(e.getDigest.toByteArray()))(breakOut),
|
e.getKey → AkkaByteString(e.getDigest.toByteArray())).toMap,
|
||||||
status.getChunk, status.getTotChunks)
|
status.getChunk, status.getTotChunks)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -287,8 +288,8 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
||||||
private def gossipFromBinary(bytes: Array[Byte]): Gossip = {
|
private def gossipFromBinary(bytes: Array[Byte]): Gossip = {
|
||||||
val gossip = dm.Gossip.parseFrom(decompress(bytes))
|
val gossip = dm.Gossip.parseFrom(decompress(bytes))
|
||||||
Gossip(
|
Gossip(
|
||||||
gossip.getEntriesList.asScala.map(e ⇒
|
gossip.getEntriesList.asScala.iterator.map(e ⇒
|
||||||
e.getKey → dataEnvelopeFromProto(e.getEnvelope))(breakOut),
|
e.getKey → dataEnvelopeFromProto(e.getEnvelope)).toMap,
|
||||||
sendBack = gossip.getSendBack)
|
sendBack = gossip.getSendBack)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -316,11 +317,11 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
||||||
DeltaPropagation(
|
DeltaPropagation(
|
||||||
uniqueAddressFromProto(deltaPropagation.getFromNode),
|
uniqueAddressFromProto(deltaPropagation.getFromNode),
|
||||||
reply,
|
reply,
|
||||||
deltaPropagation.getEntriesList.asScala.map { e ⇒
|
deltaPropagation.getEntriesList.asScala.iterator.map { e ⇒
|
||||||
val fromSeqNr = e.getFromSeqNr
|
val fromSeqNr = e.getFromSeqNr
|
||||||
val toSeqNr = if (e.hasToSeqNr) e.getToSeqNr else fromSeqNr
|
val toSeqNr = if (e.hasToSeqNr) e.getToSeqNr else fromSeqNr
|
||||||
e.getKey → Delta(dataEnvelopeFromProto(e.getEnvelope), fromSeqNr, toSeqNr)
|
e.getKey → Delta(dataEnvelopeFromProto(e.getEnvelope), fromSeqNr, toSeqNr)
|
||||||
}(breakOut))
|
}.toMap)
|
||||||
}
|
}
|
||||||
|
|
||||||
private def getToProto(get: Get[_]): dm.Get = {
|
private def getToProto(get: Get[_]): dm.Get = {
|
||||||
|
|
@ -482,7 +483,7 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
||||||
if (pruningEntries.isEmpty)
|
if (pruningEntries.isEmpty)
|
||||||
Map.empty
|
Map.empty
|
||||||
else
|
else
|
||||||
pruningEntries.asScala.map { pruningEntry ⇒
|
pruningEntries.asScala.iterator.map { pruningEntry ⇒
|
||||||
val state =
|
val state =
|
||||||
if (pruningEntry.getPerformed) {
|
if (pruningEntry.getPerformed) {
|
||||||
// for wire compatibility with Akka 2.4.x
|
// for wire compatibility with Akka 2.4.x
|
||||||
|
|
@ -491,10 +492,10 @@ class ReplicatorMessageSerializer(val system: ExtendedActorSystem)
|
||||||
} else
|
} else
|
||||||
PruningState.PruningInitialized(
|
PruningState.PruningInitialized(
|
||||||
uniqueAddressFromProto(pruningEntry.getOwnerAddress),
|
uniqueAddressFromProto(pruningEntry.getOwnerAddress),
|
||||||
pruningEntry.getSeenList.asScala.map(addressFromProto)(breakOut))
|
pruningEntry.getSeenList.asScala.iterator.map(addressFromProto).to(immutable.Set))
|
||||||
val removed = uniqueAddressFromProto(pruningEntry.getRemovedAddress)
|
val removed = uniqueAddressFromProto(pruningEntry.getRemovedAddress)
|
||||||
removed → state
|
removed → state
|
||||||
}(breakOut)
|
}.toMap
|
||||||
}
|
}
|
||||||
|
|
||||||
private def writeToProto(write: Write): dm.Write =
|
private def writeToProto(write: Write): dm.Write =
|
||||||
|
|
|
||||||
|
|
@ -11,7 +11,6 @@ import java.util.zip.GZIPOutputStream
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
import scala.collection.immutable.TreeMap
|
import scala.collection.immutable.TreeMap
|
||||||
import scala.collection.JavaConverters._
|
import scala.collection.JavaConverters._
|
||||||
import scala.collection.breakOut
|
|
||||||
import akka.actor.ActorRef
|
import akka.actor.ActorRef
|
||||||
import akka.actor.Address
|
import akka.actor.Address
|
||||||
import akka.actor.ExtendedActorSystem
|
import akka.actor.ExtendedActorSystem
|
||||||
|
|
@ -21,6 +20,7 @@ import akka.serialization._
|
||||||
import akka.protobuf.ByteString
|
import akka.protobuf.ByteString
|
||||||
import akka.protobuf.MessageLite
|
import akka.protobuf.MessageLite
|
||||||
import akka.cluster.ddata.VersionVector
|
import akka.cluster.ddata.VersionVector
|
||||||
|
import akka.util.ccompat._
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Some useful serialization helper methods.
|
* Some useful serialization helper methods.
|
||||||
|
|
@ -124,8 +124,8 @@ trait SerializationSupport {
|
||||||
else if (entries.size == 1)
|
else if (entries.size == 1)
|
||||||
VersionVector(uniqueAddressFromProto(entries.get(0).getNode), entries.get(0).getVersion)
|
VersionVector(uniqueAddressFromProto(entries.get(0).getNode), entries.get(0).getVersion)
|
||||||
else {
|
else {
|
||||||
val versions: TreeMap[UniqueAddress, Long] = versionVector.getEntriesList.asScala.map(entry ⇒
|
val versions: TreeMap[UniqueAddress, Long] = scala.collection.immutable.TreeMap.from(versionVector.getEntriesList.asScala.iterator.map(entry ⇒
|
||||||
uniqueAddressFromProto(entry.getNode) → entry.getVersion)(breakOut)
|
uniqueAddressFromProto(entry.getNode) → entry.getVersion))
|
||||||
VersionVector(versions)
|
VersionVector(versions)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -136,7 +136,7 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig)
|
||||||
expectTerminated(r)
|
expectTerminated(r)
|
||||||
|
|
||||||
var r2: ActorRef = null
|
var r2: ActorRef = null
|
||||||
awaitAssert(r2 = newReplicator()) // try until name is free
|
awaitAssert { r2 = newReplicator() } // try until name is free
|
||||||
|
|
||||||
// note that it will stash the commands until loading completed
|
// note that it will stash the commands until loading completed
|
||||||
r2 ! Get(KeyA, ReadLocal)
|
r2 ! Get(KeyA, ReadLocal)
|
||||||
|
|
@ -184,7 +184,7 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig)
|
||||||
expectTerminated(r)
|
expectTerminated(r)
|
||||||
|
|
||||||
var r2: ActorRef = null
|
var r2: ActorRef = null
|
||||||
awaitAssert(r2 = newReplicator()) // try until name is free
|
awaitAssert { r2 = newReplicator() } // try until name is free
|
||||||
awaitAssert {
|
awaitAssert {
|
||||||
r2 ! GetKeyIds
|
r2 ! GetKeyIds
|
||||||
expectMsgType[GetKeyIdsResult].keyIds should !==(Set.empty[String])
|
expectMsgType[GetKeyIdsResult].keyIds should !==(Set.empty[String])
|
||||||
|
|
@ -221,7 +221,7 @@ abstract class DurableDataSpec(multiNodeConfig: DurableDataSpecConfig)
|
||||||
expectTerminated(r)
|
expectTerminated(r)
|
||||||
|
|
||||||
var r2: ActorRef = null
|
var r2: ActorRef = null
|
||||||
awaitAssert(r2 = newReplicator()) // try until name is free
|
awaitAssert { r2 = newReplicator() } // try until name is free
|
||||||
awaitAssert {
|
awaitAssert {
|
||||||
r2 ! GetKeyIds
|
r2 ! GetKeyIds
|
||||||
expectMsgType[GetKeyIdsResult].keyIds should !==(Set.empty[String])
|
expectMsgType[GetKeyIdsResult].keyIds should !==(Set.empty[String])
|
||||||
|
|
|
||||||
|
|
@ -16,6 +16,7 @@ import akka.actor.ActorSystem
|
||||||
import akka.actor.ActorRef
|
import akka.actor.ActorRef
|
||||||
import scala.concurrent.Await
|
import scala.concurrent.Await
|
||||||
import akka.cluster.MemberStatus
|
import akka.cluster.MemberStatus
|
||||||
|
import akka.util.ccompat.imm._
|
||||||
|
|
||||||
object DurablePruningSpec extends MultiNodeConfig {
|
object DurablePruningSpec extends MultiNodeConfig {
|
||||||
val first = role("first")
|
val first = role("first")
|
||||||
|
|
@ -76,9 +77,9 @@ class DurablePruningSpec extends MultiNodeSpec(DurablePruningSpec) with STMultiN
|
||||||
Cluster(sys2).join(node(first).address)
|
Cluster(sys2).join(node(first).address)
|
||||||
awaitAssert({
|
awaitAssert({
|
||||||
Cluster(system).state.members.size should ===(4)
|
Cluster(system).state.members.size should ===(4)
|
||||||
Cluster(system).state.members.map(_.status) should ===(Set(MemberStatus.Up))
|
Cluster(system).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
|
||||||
Cluster(sys2).state.members.size should ===(4)
|
Cluster(sys2).state.members.size should ===(4)
|
||||||
Cluster(sys2).state.members.map(_.status) should ===(Set(MemberStatus.Up))
|
Cluster(sys2).state.members.unsorted.map(_.status) should ===(Set(MemberStatus.Up))
|
||||||
}, 10.seconds)
|
}, 10.seconds)
|
||||||
enterBarrier("joined")
|
enterBarrier("joined")
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -21,8 +21,8 @@ object DeltaPropagationSelectorSpec {
|
||||||
override val allNodes: Vector[Address]) extends DeltaPropagationSelector {
|
override val allNodes: Vector[Address]) extends DeltaPropagationSelector {
|
||||||
override val gossipIntervalDivisor = 5
|
override val gossipIntervalDivisor = 5
|
||||||
override def createDeltaPropagation(deltas: Map[KeyId, (ReplicatedData, Long, Long)]): DeltaPropagation =
|
override def createDeltaPropagation(deltas: Map[KeyId, (ReplicatedData, Long, Long)]): DeltaPropagation =
|
||||||
DeltaPropagation(selfUniqueAddress, false, deltas.mapValues {
|
DeltaPropagation(selfUniqueAddress, false, deltas.map {
|
||||||
case (d, fromSeqNr, toSeqNr) ⇒ Delta(DataEnvelope(d), fromSeqNr, toSeqNr)
|
case (key, (d, fromSeqNr, toSeqNr)) ⇒ (key, Delta(DataEnvelope(d), fromSeqNr, toSeqNr))
|
||||||
})
|
})
|
||||||
override def maxDeltaSize: Int = 10
|
override def maxDeltaSize: Int = 10
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -24,8 +24,8 @@ class LWWMapSpec extends WordSpec with Matchers {
|
||||||
}
|
}
|
||||||
|
|
||||||
"be able to have its entries correctly merged with another LWWMap with other entries" in {
|
"be able to have its entries correctly merged with another LWWMap with other entries" in {
|
||||||
val m1 = LWWMap.empty.put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int])
|
val m1 = LWWMap.empty[String, Int].put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int])
|
||||||
val m2 = LWWMap.empty.put(node2, "c", 3, defaultClock[Int])
|
val m2 = LWWMap.empty[String, Int].put(node2, "c", 3, defaultClock[Int])
|
||||||
|
|
||||||
// merge both ways
|
// merge both ways
|
||||||
val expected = Map("a" → 1, "b" → 2, "c" → 3)
|
val expected = Map("a" → 1, "b" → 2, "c" → 3)
|
||||||
|
|
@ -34,8 +34,8 @@ class LWWMapSpec extends WordSpec with Matchers {
|
||||||
}
|
}
|
||||||
|
|
||||||
"be able to remove entry" in {
|
"be able to remove entry" in {
|
||||||
val m1 = LWWMap.empty.put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int])
|
val m1 = LWWMap.empty[String, Int].put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int])
|
||||||
val m2 = LWWMap.empty.put(node2, "c", 3, defaultClock[Int])
|
val m2 = LWWMap.empty[String, Int].put(node2, "c", 3, defaultClock[Int])
|
||||||
|
|
||||||
val merged1 = m1 merge m2
|
val merged1 = m1 merge m2
|
||||||
|
|
||||||
|
|
@ -48,8 +48,8 @@ class LWWMapSpec extends WordSpec with Matchers {
|
||||||
}
|
}
|
||||||
|
|
||||||
"be able to work with deltas" in {
|
"be able to work with deltas" in {
|
||||||
val m1 = LWWMap.empty.put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int])
|
val m1 = LWWMap.empty[String, Int].put(node1, "a", 1, defaultClock[Int]).put(node1, "b", 2, defaultClock[Int])
|
||||||
val m2 = LWWMap.empty.put(node2, "c", 3, defaultClock[Int])
|
val m2 = LWWMap.empty[String, Int].put(node2, "c", 3, defaultClock[Int])
|
||||||
|
|
||||||
val expected = Map("a" → 1, "b" → 2, "c" → 3)
|
val expected = Map("a" → 1, "b" → 2, "c" → 3)
|
||||||
(m1 merge m2).entries should be(expected)
|
(m1 merge m2).entries should be(expected)
|
||||||
|
|
@ -69,7 +69,7 @@ class LWWMapSpec extends WordSpec with Matchers {
|
||||||
}
|
}
|
||||||
|
|
||||||
"have unapply extractor" in {
|
"have unapply extractor" in {
|
||||||
val m1 = LWWMap.empty.put(node1, "a", 1L, defaultClock[Long])
|
val m1 = LWWMap.empty[String, Long].put(node1, "a", 1L, defaultClock[Long])
|
||||||
val LWWMap(entries1) = m1
|
val LWWMap(entries1) = m1
|
||||||
val entries2: Map[String, Long] = entries1
|
val entries2: Map[String, Long] = entries1
|
||||||
Changed(LWWMapKey[String, Long]("key"))(m1) match {
|
Changed(LWWMapKey[String, Long]("key"))(m1) match {
|
||||||
|
|
|
||||||
|
|
@ -368,21 +368,7 @@ you can also parallelize it by chunking your futures into sub-sequences and redu
|
||||||
## Callbacks
|
## Callbacks
|
||||||
|
|
||||||
Sometimes you just want to listen to a `Future` being completed, and react to that not by creating a new `Future`, but by side-effecting.
|
Sometimes you just want to listen to a `Future` being completed, and react to that not by creating a new `Future`, but by side-effecting.
|
||||||
For this `Future` supports `onComplete`, `onSuccess` and `onFailure`, of which the last two are specializations of the first.
|
For this, `Future` supports `onComplete`:
|
||||||
|
|
||||||
Scala
|
|
||||||
: @@snip [FutureDocSpec.scala](/akka-docs/src/test/scala/docs/future/FutureDocSpec.scala) { #onSuccess }
|
|
||||||
|
|
||||||
Java
|
|
||||||
: @@snip [FutureDocTest.java](/akka-docs/src/test/java/jdocs/future/FutureDocTest.java) { #onSuccess }
|
|
||||||
|
|
||||||
|
|
||||||
Scala
|
|
||||||
: @@snip [FutureDocSpec.scala](/akka-docs/src/test/scala/docs/future/FutureDocSpec.scala) { #onFailure }
|
|
||||||
|
|
||||||
Java
|
|
||||||
: @@snip [FutureDocTest.java](/akka-docs/src/test/java/jdocs/future/FutureDocTest.java) { #onFailure }
|
|
||||||
|
|
||||||
|
|
||||||
Scala
|
Scala
|
||||||
: @@snip [FutureDocSpec.scala](/akka-docs/src/test/scala/docs/future/FutureDocSpec.scala) { #onComplete }
|
: @@snip [FutureDocSpec.scala](/akka-docs/src/test/scala/docs/future/FutureDocSpec.scala) { #onComplete }
|
||||||
|
|
|
||||||
|
|
@ -18,14 +18,15 @@ import akka.util.Timeout;
|
||||||
|
|
||||||
//#imports2
|
//#imports2
|
||||||
import java.time.Duration;
|
import java.time.Duration;
|
||||||
import akka.japi.Function;
|
|
||||||
|
|
||||||
import java.util.concurrent.*;
|
import java.util.concurrent.*;
|
||||||
|
|
||||||
|
import scala.util.Try;
|
||||||
|
|
||||||
|
import akka.japi.Function;
|
||||||
|
|
||||||
import static akka.dispatch.Futures.future;
|
import static akka.dispatch.Futures.future;
|
||||||
import static java.util.concurrent.TimeUnit.SECONDS;
|
import static java.util.concurrent.TimeUnit.SECONDS;
|
||||||
|
|
||||||
|
|
||||||
//#imports2
|
//#imports2
|
||||||
|
|
||||||
//#imports3
|
//#imports3
|
||||||
|
|
@ -283,7 +284,7 @@ public class FutureDocTest extends AbstractJavaTest {
|
||||||
}
|
}
|
||||||
}, system.dispatcher());
|
}, system.dispatcher());
|
||||||
|
|
||||||
f.onSuccess(new PrintResult<String>(), system.dispatcher());
|
f.onComplete(new PrintResult<Try<String>>(), system.dispatcher());
|
||||||
//#future-eval
|
//#future-eval
|
||||||
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
||||||
String result = (String) Await.result(f, timeout.duration());
|
String result = (String) Await.result(f, timeout.duration());
|
||||||
|
|
@ -307,7 +308,7 @@ public class FutureDocTest extends AbstractJavaTest {
|
||||||
}
|
}
|
||||||
}, ec);
|
}, ec);
|
||||||
|
|
||||||
f2.onSuccess(new PrintResult<Integer>(), system.dispatcher());
|
f2.onComplete(new PrintResult<Try<Integer>>(), system.dispatcher());
|
||||||
//#map
|
//#map
|
||||||
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
||||||
int result = Await.result(f2, timeout.duration());
|
int result = Await.result(f2, timeout.duration());
|
||||||
|
|
@ -335,7 +336,7 @@ public class FutureDocTest extends AbstractJavaTest {
|
||||||
}
|
}
|
||||||
}, ec);
|
}, ec);
|
||||||
|
|
||||||
f2.onSuccess(new PrintResult<Integer>(), system.dispatcher());
|
f2.onComplete(new PrintResult<Try<Integer>>(), system.dispatcher());
|
||||||
//#flat-map
|
//#flat-map
|
||||||
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
||||||
int result = Await.result(f2, timeout.duration());
|
int result = Await.result(f2, timeout.duration());
|
||||||
|
|
@ -367,7 +368,7 @@ public class FutureDocTest extends AbstractJavaTest {
|
||||||
}
|
}
|
||||||
}, ec);
|
}, ec);
|
||||||
|
|
||||||
futureSum.onSuccess(new PrintResult<Long>(), system.dispatcher());
|
futureSum.onComplete(new PrintResult<Try<Long>>(), system.dispatcher());
|
||||||
//#sequence
|
//#sequence
|
||||||
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
||||||
long result = Await.result(futureSum, timeout.duration());
|
long result = Await.result(futureSum, timeout.duration());
|
||||||
|
|
@ -393,7 +394,7 @@ public class FutureDocTest extends AbstractJavaTest {
|
||||||
}, ec);
|
}, ec);
|
||||||
|
|
||||||
//Returns the sequence of strings as upper case
|
//Returns the sequence of strings as upper case
|
||||||
futureResult.onSuccess(new PrintResult<Iterable<String>>(), system.dispatcher());
|
futureResult.onComplete(new PrintResult<Try<Iterable<String>>>(), system.dispatcher());
|
||||||
//#traverse
|
//#traverse
|
||||||
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
||||||
Iterable<String> result = Await.result(futureResult, timeout.duration());
|
Iterable<String> result = Await.result(futureResult, timeout.duration());
|
||||||
|
|
@ -420,7 +421,7 @@ public class FutureDocTest extends AbstractJavaTest {
|
||||||
}
|
}
|
||||||
}, ec);
|
}, ec);
|
||||||
|
|
||||||
resultFuture.onSuccess(new PrintResult<String>(), system.dispatcher());
|
resultFuture.onComplete(new PrintResult<Try<String>>(), system.dispatcher());
|
||||||
//#fold
|
//#fold
|
||||||
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
||||||
String result = Await.result(resultFuture, timeout.duration());
|
String result = Await.result(resultFuture, timeout.duration());
|
||||||
|
|
@ -445,7 +446,7 @@ public class FutureDocTest extends AbstractJavaTest {
|
||||||
}
|
}
|
||||||
}, ec);
|
}, ec);
|
||||||
|
|
||||||
resultFuture.onSuccess(new PrintResult<Object>(), system.dispatcher());
|
resultFuture.onComplete(new PrintResult<Try<Object>>(), system.dispatcher());
|
||||||
//#reduce
|
//#reduce
|
||||||
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
||||||
Object result = Await.result(resultFuture, timeout.duration());
|
Object result = Await.result(resultFuture, timeout.duration());
|
||||||
|
|
@ -545,7 +546,7 @@ public class FutureDocTest extends AbstractJavaTest {
|
||||||
}
|
}
|
||||||
}, ec);
|
}, ec);
|
||||||
|
|
||||||
future.onSuccess(new PrintResult<Integer>(), system.dispatcher());
|
future.onComplete(new PrintResult<Try<Integer>>(), system.dispatcher());
|
||||||
//#recover
|
//#recover
|
||||||
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
||||||
int result = Await.result(future, timeout.duration());
|
int result = Await.result(future, timeout.duration());
|
||||||
|
|
@ -574,7 +575,7 @@ public class FutureDocTest extends AbstractJavaTest {
|
||||||
}
|
}
|
||||||
}, ec);
|
}, ec);
|
||||||
|
|
||||||
future.onSuccess(new PrintResult<Integer>(), system.dispatcher());
|
future.onComplete(new PrintResult<Try<Integer>>(), system.dispatcher());
|
||||||
//#try-recover
|
//#try-recover
|
||||||
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
||||||
int result = Await.result(future, timeout.duration());
|
int result = Await.result(future, timeout.duration());
|
||||||
|
|
@ -582,42 +583,10 @@ public class FutureDocTest extends AbstractJavaTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void useOnSuccessOnFailureAndOnComplete() throws Exception {
|
public void useOnOnComplete() throws Exception {
|
||||||
{
|
{
|
||||||
Future<String> future = Futures.successful("foo");
|
Future<String> future = Futures.successful("foo");
|
||||||
|
|
||||||
//#onSuccess
|
|
||||||
final ExecutionContext ec = system.dispatcher();
|
|
||||||
|
|
||||||
future.onSuccess(new OnSuccess<String>() {
|
|
||||||
public void onSuccess(String result) {
|
|
||||||
if ("bar" == result) {
|
|
||||||
//Do something if it resulted in "bar"
|
|
||||||
} else {
|
|
||||||
//Do something if it was some other String
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}, ec);
|
|
||||||
//#onSuccess
|
|
||||||
}
|
|
||||||
{
|
|
||||||
Future<String> future = Futures.failed(new IllegalStateException("OHNOES"));
|
|
||||||
//#onFailure
|
|
||||||
final ExecutionContext ec = system.dispatcher();
|
|
||||||
|
|
||||||
future.onFailure(new OnFailure() {
|
|
||||||
public void onFailure(Throwable failure) {
|
|
||||||
if (failure instanceof IllegalStateException) {
|
|
||||||
//Do something if it was this particular failure
|
|
||||||
} else {
|
|
||||||
//Do something if it was some other failure
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}, ec);
|
|
||||||
//#onFailure
|
|
||||||
}
|
|
||||||
{
|
|
||||||
Future<String> future = Futures.successful("foo");
|
|
||||||
//#onComplete
|
//#onComplete
|
||||||
final ExecutionContext ec = system.dispatcher();
|
final ExecutionContext ec = system.dispatcher();
|
||||||
|
|
||||||
|
|
@ -648,7 +617,7 @@ public class FutureDocTest extends AbstractJavaTest {
|
||||||
}
|
}
|
||||||
}, ec);
|
}, ec);
|
||||||
|
|
||||||
future3.onSuccess(new PrintResult<String>(), system.dispatcher());
|
future3.onComplete(new PrintResult<Try<String>>(), system.dispatcher());
|
||||||
//#zip
|
//#zip
|
||||||
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
||||||
String result = Await.result(future3, timeout.duration());
|
String result = Await.result(future3, timeout.duration());
|
||||||
|
|
@ -662,7 +631,7 @@ public class FutureDocTest extends AbstractJavaTest {
|
||||||
Future<String> future3 = Futures.successful("bar");
|
Future<String> future3 = Futures.successful("bar");
|
||||||
// Will have "bar" in this case
|
// Will have "bar" in this case
|
||||||
Future<String> future4 = future1.fallbackTo(future2).fallbackTo(future3);
|
Future<String> future4 = future1.fallbackTo(future2).fallbackTo(future3);
|
||||||
future4.onSuccess(new PrintResult<String>(), system.dispatcher());
|
future4.onComplete(new PrintResult<Try<String>>(), system.dispatcher());
|
||||||
//#fallback-to
|
//#fallback-to
|
||||||
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
Timeout timeout = Timeout.create(Duration.ofSeconds(5));
|
||||||
String result = Await.result(future4, timeout.duration());
|
String result = Await.result(future4, timeout.duration());
|
||||||
|
|
|
||||||
|
|
@ -197,7 +197,7 @@ public class PersistenceQueryDocTest {
|
||||||
|
|
||||||
public akka.stream.scaladsl.Source<RichEvent, QueryMetadata> byTagsWithMeta(
|
public akka.stream.scaladsl.Source<RichEvent, QueryMetadata> byTagsWithMeta(
|
||||||
scala.collection.Set<String> tags) {
|
scala.collection.Set<String> tags) {
|
||||||
Set<String> jTags = scala.collection.JavaConversions.setAsJavaSet(tags);
|
Set<String> jTags = scala.collection.JavaConverters.setAsJavaSetConverter(tags).asJava();
|
||||||
return javadslReadJournal.byTagsWithMeta(jTags).asScala();
|
return javadslReadJournal.byTagsWithMeta(jTags).asScala();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -5,6 +5,7 @@
|
||||||
package scala.docs.cluster
|
package scala.docs.cluster
|
||||||
|
|
||||||
import language.postfixOps
|
import language.postfixOps
|
||||||
|
import scala.util.Success
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import akka.actor.Actor
|
import akka.actor.Actor
|
||||||
import akka.actor.ActorRef
|
import akka.actor.ActorRef
|
||||||
|
|
@ -55,9 +56,8 @@ object TransformationFrontend {
|
||||||
import system.dispatcher
|
import system.dispatcher
|
||||||
system.scheduler.schedule(2.seconds, 2.seconds) {
|
system.scheduler.schedule(2.seconds, 2.seconds) {
|
||||||
implicit val timeout = Timeout(5 seconds)
|
implicit val timeout = Timeout(5 seconds)
|
||||||
(frontend ? TransformationJob("hello-" + counter.incrementAndGet())) onSuccess {
|
(frontend ? TransformationJob("hello-" + counter.incrementAndGet()))
|
||||||
case result ⇒ println(result)
|
.foreach { result ⇒ println(result) }
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
Some files were not shown because too many files have changed in this diff Show more
Loading…
Add table
Add a link
Reference in a new issue