Various scala-2.13.0-M5 fixes

fix akka-actor-tests compile errors

some tests still fail though

Fix test failures in akka-actor-test

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

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

akka-remote scalafix changes

Fix shutdownAll compile error

test:akka-remote scalafix changes

akka-multi-node-testkit scalafix

Fix akka-remote-tests multi-jvm compile errors

akka-stream-tests/test:scalafix

Fix test:akka-stream-tests

Crude implementation of ByteString.map

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

akka-actor-typed-tests compile and succeed

scalafix akka-camel

scalafix akka-cluster

akka-cluster compile & test

scalafix akka-cluster-metrics

Fix akka-cluster-metrics

scalafix akka-cluster-tools

akka-cluster-tools compile and test

scalafix akka-distributed-data

akka-distributed-data fixes

scalafix akka-persistence

scalafix akka-cluster-sharding

fix akka-cluster-sharding

scalafix akka-contrib

Fix akka-cluster-sharding-typed test

scalafix akka-docs

Use scala-stm 0.9 (released for M5)

akka-docs

Remove dependency on collections-compat

Cherry-pick the relevant constructs to our own
private utils

Shorten 'scala.collections.immutable' by importing it

Duplicate 'immutable' imports

Use 'foreach' on futures

Replace MapLike with regular Map

Internal API markers

Simplify ccompat by moving PackageShared into object

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

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

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

Make more things more private

Copyright headers

Use 'unsorted' to go from SortedSet to Set

Duplicate import

Use onComplete rather than failed.foreach

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

View file

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

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