Merge pull request #22206 from akka/wip-21423-remove-deprecations-patriknw
remove deprecations, #21423
This commit is contained in:
commit
94e40460a4
32 changed files with 105 additions and 361 deletions
|
|
@ -197,30 +197,26 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend
|
||||||
import system.dispatcher
|
import system.dispatcher
|
||||||
system2.scheduler.scheduleOnce(200.millis.dilated) { system2.terminate() }
|
system2.scheduler.scheduleOnce(200.millis.dilated) { system2.terminate() }
|
||||||
|
|
||||||
system2.awaitTermination(5 seconds)
|
|
||||||
Await.ready(system2.whenTerminated, 5 seconds)
|
Await.ready(system2.whenTerminated, 5 seconds)
|
||||||
callbackWasRun should ===(true)
|
callbackWasRun should ===(true)
|
||||||
}
|
}
|
||||||
|
|
||||||
"return isTerminated status correctly" in {
|
"return isTerminated status correctly" in {
|
||||||
val system = ActorSystem().asInstanceOf[ActorSystemImpl]
|
val system = ActorSystem().asInstanceOf[ActorSystemImpl]
|
||||||
system.isTerminated should ===(false)
|
|
||||||
val wt = system.whenTerminated
|
val wt = system.whenTerminated
|
||||||
wt.isCompleted should ===(false)
|
wt.isCompleted should ===(false)
|
||||||
val f = system.terminate()
|
val f = system.terminate()
|
||||||
val terminated = Await.result(wt, 10 seconds)
|
val terminated = Await.result(wt, 10 seconds)
|
||||||
|
system.whenTerminated.isCompleted should ===(true)
|
||||||
terminated.actor should ===(system.provider.rootGuardian)
|
terminated.actor should ===(system.provider.rootGuardian)
|
||||||
terminated.addressTerminated should ===(true)
|
terminated.addressTerminated should ===(true)
|
||||||
terminated.existenceConfirmed should ===(true)
|
terminated.existenceConfirmed should ===(true)
|
||||||
terminated should be theSameInstanceAs Await.result(f, 10 seconds)
|
terminated should be theSameInstanceAs Await.result(f, 10 seconds)
|
||||||
system.awaitTermination(10 seconds)
|
|
||||||
system.isTerminated should ===(true)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
"throw RejectedExecutionException when shutdown" in {
|
"throw RejectedExecutionException when shutdown" in {
|
||||||
val system2 = ActorSystem("AwaitTermination", AkkaSpec.testConf)
|
val system2 = ActorSystem("AwaitTermination", AkkaSpec.testConf)
|
||||||
Await.ready(system2.terminate(), 10 seconds)
|
Await.ready(system2.terminate(), 10 seconds)
|
||||||
system2.awaitTermination(10 seconds)
|
|
||||||
|
|
||||||
intercept[RejectedExecutionException] {
|
intercept[RejectedExecutionException] {
|
||||||
system2.registerOnTermination { println("IF YOU SEE THIS THEN THERE'S A BUG HERE") }
|
system2.registerOnTermination { println("IF YOU SEE THIS THEN THERE'S A BUG HERE") }
|
||||||
|
|
|
||||||
|
|
@ -44,7 +44,7 @@ class AddressTerminatedTopicBenchSpec extends AkkaSpec("akka.loglevel=INFO") {
|
||||||
shutdown(sys, 10.seconds, verifySystemShutdown = true)
|
shutdown(sys, 10.seconds, verifySystemShutdown = true)
|
||||||
log.info("Stopping {} actors took {} ms", num, (System.nanoTime() - t2).nanos.toMillis)
|
log.info("Stopping {} actors took {} ms", num, (System.nanoTime() - t2).nanos.toMillis)
|
||||||
} finally {
|
} finally {
|
||||||
if (!sys.isTerminated) shutdown(sys)
|
shutdown(sys)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -58,9 +58,6 @@ object ActorPath {
|
||||||
case _ ⇒ throw new MalformedURLException("cannot parse as ActorPath: " + s)
|
case _ ⇒ throw new MalformedURLException("cannot parse as ActorPath: " + s)
|
||||||
}
|
}
|
||||||
|
|
||||||
@deprecated("Use `isValidPathElement` instead", since = "2.3.8")
|
|
||||||
val ElementRegex = """(?:[-\w:@&=+,.!~*'_;]|%\p{XDigit}{2})(?:[-\w:@&=+,.!~*'$_;]|%\p{XDigit}{2})*""".r
|
|
||||||
|
|
||||||
private final val ValidSymbols = """-_.*$+:@&=,!~';"""
|
private final val ValidSymbols = """-_.*$+:@&=,!~';"""
|
||||||
|
|
||||||
private final val ValidPathCode = -1
|
private final val ValidPathCode = -1
|
||||||
|
|
|
||||||
|
|
@ -3,6 +3,7 @@
|
||||||
*/
|
*/
|
||||||
package akka.actor
|
package akka.actor
|
||||||
|
|
||||||
|
import scala.language.implicitConversions
|
||||||
import java.util.concurrent.CompletionStage
|
import java.util.concurrent.CompletionStage
|
||||||
|
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
|
|
@ -231,7 +232,7 @@ object ActorSelection {
|
||||||
matchingChildren.foreach(_.tell(sel.msg, sender))
|
matchingChildren.foreach(_.tell(sel.msg, sender))
|
||||||
} else {
|
} else {
|
||||||
val matchingChildren = chldr.filter(c ⇒ p.pattern.matcher(c.path.name).matches)
|
val matchingChildren = chldr.filter(c ⇒ p.pattern.matcher(c.path.name).matches)
|
||||||
// don't send to emptyRef after wildcard fan-out
|
// don't send to emptyRef after wildcard fan-out
|
||||||
if (matchingChildren.isEmpty && !sel.wildcardFanOut)
|
if (matchingChildren.isEmpty && !sel.wildcardFanOut)
|
||||||
emptyRef.tell(sel, sender)
|
emptyRef.tell(sel, sender)
|
||||||
else {
|
else {
|
||||||
|
|
|
||||||
|
|
@ -523,42 +523,6 @@ abstract class ActorSystem extends ActorRefFactory {
|
||||||
*/
|
*/
|
||||||
def registerOnTermination(code: Runnable): Unit
|
def registerOnTermination(code: Runnable): Unit
|
||||||
|
|
||||||
/**
|
|
||||||
* Block current thread until the system has been shutdown, or the specified
|
|
||||||
* timeout has elapsed. This will block until after all on termination
|
|
||||||
* callbacks have been run.
|
|
||||||
*
|
|
||||||
* Throws TimeoutException in case of timeout.
|
|
||||||
*/
|
|
||||||
@deprecated("Use Await.result(whenTerminated, timeout) instead", "2.4")
|
|
||||||
def awaitTermination(timeout: Duration): Unit
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Block current thread until the system has been shutdown. This will
|
|
||||||
* block until after all on termination callbacks have been run.
|
|
||||||
*/
|
|
||||||
@deprecated("Use Await.result(whenTerminated, Duration.Inf) instead", "2.4")
|
|
||||||
def awaitTermination(): Unit
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Stop this actor system. This will stop the guardian actor, which in turn
|
|
||||||
* will recursively stop all its child actors, then the system guardian
|
|
||||||
* (below which the logging actors reside) and the execute all registered
|
|
||||||
* termination handlers (see [[ActorSystem#registerOnTermination]]).
|
|
||||||
*/
|
|
||||||
@deprecated("Use the terminate() method instead", "2.4")
|
|
||||||
def shutdown(): Unit
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Query the termination status: if it returns true, all callbacks have run
|
|
||||||
* and the ActorSystem has been fully stopped, i.e.
|
|
||||||
* `awaitTermination(0 seconds)` would return normally. If this method
|
|
||||||
* returns `false`, the status is actually unknown, since it might have
|
|
||||||
* changed since you queried it.
|
|
||||||
*/
|
|
||||||
@deprecated("Use the whenTerminated method instead.", "2.4")
|
|
||||||
def isTerminated: Boolean
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Terminates this actor system. This will stop the guardian actor, which in turn
|
* Terminates this actor system. This will stop the guardian actor, which in turn
|
||||||
* will recursively stop all its child actors, then the system guardian
|
* will recursively stop all its child actors, then the system guardian
|
||||||
|
|
@ -820,11 +784,6 @@ private[akka] class ActorSystemImpl(
|
||||||
def start(): this.type = _start
|
def start(): this.type = _start
|
||||||
def registerOnTermination[T](code: ⇒ T) { registerOnTermination(new Runnable { def run = code }) }
|
def registerOnTermination[T](code: ⇒ T) { registerOnTermination(new Runnable { def run = code }) }
|
||||||
def registerOnTermination(code: Runnable) { terminationCallbacks.add(code) }
|
def registerOnTermination(code: Runnable) { terminationCallbacks.add(code) }
|
||||||
override def awaitTermination(timeout: Duration) { Await.ready(whenTerminated, timeout) }
|
|
||||||
override def awaitTermination() = awaitTermination(Duration.Inf)
|
|
||||||
override def isTerminated = whenTerminated.isCompleted
|
|
||||||
|
|
||||||
override def shutdown(): Unit = terminate()
|
|
||||||
|
|
||||||
override def terminate(): Future[Terminated] = {
|
override def terminate(): Future[Terminated] = {
|
||||||
if (!settings.LogDeadLettersDuringShutdown) logDeadLetterListener foreach stop
|
if (!settings.LogDeadLettersDuringShutdown) logDeadLetterListener foreach stop
|
||||||
|
|
|
||||||
|
|
@ -412,112 +412,3 @@ trait ManagedActorClassification { this: ActorEventBus with ActorClassifier ⇒
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Maps ActorRefs to ActorRefs to form an EventBus where ActorRefs can listen to other ActorRefs
|
|
||||||
*/
|
|
||||||
@deprecated("Use Managed ActorClassification instead", "2.4")
|
|
||||||
trait ActorClassification { this: ActorEventBus with ActorClassifier ⇒
|
|
||||||
import java.util.concurrent.ConcurrentHashMap
|
|
||||||
import scala.annotation.tailrec
|
|
||||||
private val empty = immutable.TreeSet.empty[ActorRef]
|
|
||||||
private val mappings = new ConcurrentHashMap[ActorRef, immutable.TreeSet[ActorRef]](mapSize)
|
|
||||||
|
|
||||||
@tailrec
|
|
||||||
protected final def associate(monitored: ActorRef, monitor: ActorRef): Boolean = {
|
|
||||||
val current = mappings get monitored
|
|
||||||
current match {
|
|
||||||
case null ⇒
|
|
||||||
if (monitored.isTerminated) false
|
|
||||||
else {
|
|
||||||
if (mappings.putIfAbsent(monitored, empty + monitor) ne null) associate(monitored, monitor)
|
|
||||||
else if (monitored.isTerminated) !dissociate(monitored, monitor) else true
|
|
||||||
}
|
|
||||||
case raw: immutable.TreeSet[_] ⇒
|
|
||||||
val v = raw.asInstanceOf[immutable.TreeSet[ActorRef]]
|
|
||||||
if (monitored.isTerminated) false
|
|
||||||
if (v.contains(monitor)) true
|
|
||||||
else {
|
|
||||||
val added = v + monitor
|
|
||||||
if (!mappings.replace(monitored, v, added)) associate(monitored, monitor)
|
|
||||||
else if (monitored.isTerminated) !dissociate(monitored, monitor) else true
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
protected final def dissociate(monitored: ActorRef): immutable.Iterable[ActorRef] = {
|
|
||||||
@tailrec
|
|
||||||
def dissociateAsMonitored(monitored: ActorRef): immutable.Iterable[ActorRef] = {
|
|
||||||
val current = mappings get monitored
|
|
||||||
current match {
|
|
||||||
case null ⇒ empty
|
|
||||||
case raw: immutable.TreeSet[_] ⇒
|
|
||||||
val v = raw.asInstanceOf[immutable.TreeSet[ActorRef]]
|
|
||||||
if (!mappings.remove(monitored, v)) dissociateAsMonitored(monitored)
|
|
||||||
else v
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
def dissociateAsMonitor(monitor: ActorRef): Unit = {
|
|
||||||
val i = mappings.entrySet.iterator
|
|
||||||
while (i.hasNext()) {
|
|
||||||
val entry = i.next()
|
|
||||||
val v = entry.getValue
|
|
||||||
v match {
|
|
||||||
case raw: immutable.TreeSet[_] ⇒
|
|
||||||
val monitors = raw.asInstanceOf[immutable.TreeSet[ActorRef]]
|
|
||||||
if (monitors.contains(monitor))
|
|
||||||
dissociate(entry.getKey, monitor)
|
|
||||||
case _ ⇒ //Dun care
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
try { dissociateAsMonitored(monitored) } finally { dissociateAsMonitor(monitored) }
|
|
||||||
}
|
|
||||||
|
|
||||||
@tailrec
|
|
||||||
protected final def dissociate(monitored: ActorRef, monitor: ActorRef): Boolean = {
|
|
||||||
val current = mappings get monitored
|
|
||||||
current match {
|
|
||||||
case null ⇒ false
|
|
||||||
case raw: immutable.TreeSet[_] ⇒
|
|
||||||
val v = raw.asInstanceOf[immutable.TreeSet[ActorRef]]
|
|
||||||
val removed = v - monitor
|
|
||||||
if (removed eq raw) false
|
|
||||||
else if (removed.isEmpty) {
|
|
||||||
if (!mappings.remove(monitored, v)) dissociate(monitored, monitor) else true
|
|
||||||
} else {
|
|
||||||
if (!mappings.replace(monitored, v, removed)) dissociate(monitored, monitor) else true
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns the Classifier associated with the specified Event
|
|
||||||
*/
|
|
||||||
protected def classify(event: Event): Classifier
|
|
||||||
|
|
||||||
/**
|
|
||||||
* This is a size hint for the number of Classifiers you expect to have (use powers of 2)
|
|
||||||
*/
|
|
||||||
protected def mapSize: Int
|
|
||||||
|
|
||||||
def publish(event: Event): Unit = mappings.get(classify(event)) match {
|
|
||||||
case null ⇒ ()
|
|
||||||
case some ⇒ some foreach { _ ! event }
|
|
||||||
}
|
|
||||||
|
|
||||||
def subscribe(subscriber: Subscriber, to: Classifier): Boolean =
|
|
||||||
if (subscriber eq null) throw new IllegalArgumentException("Subscriber is null")
|
|
||||||
else if (to eq null) throw new IllegalArgumentException("Classifier is null")
|
|
||||||
else associate(to, subscriber)
|
|
||||||
|
|
||||||
def unsubscribe(subscriber: Subscriber, from: Classifier): Boolean =
|
|
||||||
if (subscriber eq null) throw new IllegalArgumentException("Subscriber is null")
|
|
||||||
else if (from eq null) throw new IllegalArgumentException("Classifier is null")
|
|
||||||
else dissociate(from, subscriber)
|
|
||||||
|
|
||||||
def unsubscribe(subscriber: Subscriber): Unit =
|
|
||||||
if (subscriber eq null) throw new IllegalArgumentException("Subscriber is null")
|
|
||||||
else dissociate(subscriber)
|
|
||||||
}
|
|
||||||
|
|
|
||||||
|
|
@ -11,11 +11,6 @@ import akka.util.Subclassification
|
||||||
import java.util.concurrent.atomic.AtomicReference
|
import java.util.concurrent.atomic.AtomicReference
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
|
|
||||||
object EventStream {
|
|
||||||
@deprecated("Use explicit `system.eventStream` instead", "2.4")
|
|
||||||
implicit def fromActorSystem(system: ActorSystem) = system.eventStream
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An Akka EventStream is a pub-sub stream of events both system and user generated,
|
* An Akka EventStream is a pub-sub stream of events both system and user generated,
|
||||||
* where subscribers are ActorRefs and the channels are Classes and Events are any java.lang.Object.
|
* where subscribers are ActorRefs and the channels are Classes and Events are any java.lang.Object.
|
||||||
|
|
@ -29,9 +24,6 @@ class EventStream(sys: ActorSystem, private val debug: Boolean) extends LoggingB
|
||||||
|
|
||||||
def this(sys: ActorSystem) = this(sys, debug = false)
|
def this(sys: ActorSystem) = this(sys, debug = false)
|
||||||
|
|
||||||
@deprecated("Use constructor with ActorSystem parameter", "2.4")
|
|
||||||
def this(debug: Boolean = false) = this(sys = null, debug)
|
|
||||||
|
|
||||||
type Event = AnyRef
|
type Event = AnyRef
|
||||||
type Classifier = Class[_]
|
type Classifier = Class[_]
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -217,35 +217,3 @@ abstract class ManagedActorEventBus[E](system: ActorSystem) extends EventBus[E,
|
||||||
override def publish(event: E): Unit = bus.publish(event)
|
override def publish(event: E): Unit = bus.publish(event)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Java API: See documentation for [[akka.event.ActorClassification]]
|
|
||||||
* An EventBus where the Subscribers are ActorRefs and the Classifier is ActorRef
|
|
||||||
* Means that ActorRefs "listen" to other ActorRefs
|
|
||||||
* E is the Event type
|
|
||||||
*/
|
|
||||||
@deprecated("Use ManagedActorEventBus instead", "2.4")
|
|
||||||
abstract class ActorEventBus[E] extends EventBus[E, ActorRef, ActorRef] {
|
|
||||||
private val bus = new akka.event.ActorEventBus with akka.event.ActorClassification with akka.event.ActorClassifier {
|
|
||||||
type Event = E
|
|
||||||
|
|
||||||
override protected def mapSize: Int = ActorEventBus.this.mapSize
|
|
||||||
|
|
||||||
override protected def classify(event: E): ActorRef =
|
|
||||||
ActorEventBus.this.classify(event)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* This is a size hint for the number of Classifiers you expect to have (use powers of 2)
|
|
||||||
*/
|
|
||||||
protected def mapSize(): Int
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns the Classifier associated with the given Event
|
|
||||||
*/
|
|
||||||
protected def classify(event: E): ActorRef
|
|
||||||
|
|
||||||
override def subscribe(subscriber: ActorRef, to: ActorRef): Boolean = bus.subscribe(subscriber, to)
|
|
||||||
override def unsubscribe(subscriber: ActorRef, from: ActorRef): Boolean = bus.unsubscribe(subscriber, from)
|
|
||||||
override def unsubscribe(subscriber: ActorRef): Unit = bus.unsubscribe(subscriber)
|
|
||||||
override def publish(event: E): Unit = bus.publish(event)
|
|
||||||
}
|
|
||||||
|
|
|
||||||
|
|
@ -438,9 +438,6 @@ private[akka] final class PromiseActorRef private (val provider: ActorRefProvide
|
||||||
import AbstractPromiseActorRef.{ stateOffset, watchedByOffset }
|
import AbstractPromiseActorRef.{ stateOffset, watchedByOffset }
|
||||||
import PromiseActorRef._
|
import PromiseActorRef._
|
||||||
|
|
||||||
@deprecated("Use the full constructor", "2.4")
|
|
||||||
def this(provider: ActorRefProvider, result: Promise[Any]) = this(provider, result, "unknown")
|
|
||||||
|
|
||||||
// This is necessary for weaving the PromiseActorRef into the asked message, i.e. the replyTo pattern.
|
// This is necessary for weaving the PromiseActorRef into the asked message, i.e. the replyTo pattern.
|
||||||
@volatile var messageClassName = _mcn
|
@volatile var messageClassName = _mcn
|
||||||
|
|
||||||
|
|
@ -607,7 +604,4 @@ private[akka] object PromiseActorRef {
|
||||||
a
|
a
|
||||||
}
|
}
|
||||||
|
|
||||||
@deprecated("Use apply with messageClassName and sender parameters", "2.4")
|
|
||||||
def apply(provider: ActorRefProvider, timeout: Timeout, targetName: String): PromiseActorRef =
|
|
||||||
apply(provider, timeout, targetName, "unknown", Actor.noSender)
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -66,7 +66,7 @@ private[akka] final class BalancingRoutingLogic extends RoutingLogic {
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
final case class BalancingPool(
|
final case class BalancingPool(
|
||||||
override val nrOfInstances: Int,
|
val nrOfInstances: Int,
|
||||||
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
||||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
||||||
extends Pool {
|
extends Pool {
|
||||||
|
|
|
||||||
|
|
@ -56,7 +56,7 @@ final class BroadcastRoutingLogic extends RoutingLogic {
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
final case class BroadcastPool(
|
final case class BroadcastPool(
|
||||||
override val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
|
val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
|
||||||
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
||||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||||
override val usePoolDispatcher: Boolean = false)
|
override val usePoolDispatcher: Boolean = false)
|
||||||
|
|
@ -118,7 +118,7 @@ final case class BroadcastPool(
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
final case class BroadcastGroup(
|
final case class BroadcastGroup(
|
||||||
override val paths: immutable.Iterable[String],
|
val paths: immutable.Iterable[String],
|
||||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
||||||
extends Group {
|
extends Group {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -267,7 +267,7 @@ final case class ConsistentHashingRoutingLogic(
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
final case class ConsistentHashingPool(
|
final case class ConsistentHashingPool(
|
||||||
override val nrOfInstances: Int,
|
val nrOfInstances: Int,
|
||||||
override val resizer: Option[Resizer] = None,
|
override val resizer: Option[Resizer] = None,
|
||||||
val virtualNodesFactor: Int = 0,
|
val virtualNodesFactor: Int = 0,
|
||||||
val hashMapping: ConsistentHashingRouter.ConsistentHashMapping = ConsistentHashingRouter.emptyConsistentHashMapping,
|
val hashMapping: ConsistentHashingRouter.ConsistentHashMapping = ConsistentHashingRouter.emptyConsistentHashMapping,
|
||||||
|
|
@ -355,7 +355,7 @@ final case class ConsistentHashingPool(
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
final case class ConsistentHashingGroup(
|
final case class ConsistentHashingGroup(
|
||||||
override val paths: immutable.Iterable[String],
|
val paths: immutable.Iterable[String],
|
||||||
val virtualNodesFactor: Int = 0,
|
val virtualNodesFactor: Int = 0,
|
||||||
val hashMapping: ConsistentHashingRouter.ConsistentHashMapping = ConsistentHashingRouter.emptyConsistentHashMapping,
|
val hashMapping: ConsistentHashingRouter.ConsistentHashMapping = ConsistentHashingRouter.emptyConsistentHashMapping,
|
||||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
||||||
|
|
|
||||||
|
|
@ -57,7 +57,7 @@ final class RandomRoutingLogic extends RoutingLogic {
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
final case class RandomPool(
|
final case class RandomPool(
|
||||||
override val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
|
val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
|
||||||
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
||||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||||
override val usePoolDispatcher: Boolean = false)
|
override val usePoolDispatcher: Boolean = false)
|
||||||
|
|
@ -119,7 +119,7 @@ final case class RandomPool(
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
final case class RandomGroup(
|
final case class RandomGroup(
|
||||||
override val paths: immutable.Iterable[String],
|
val paths: immutable.Iterable[String],
|
||||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
||||||
extends Group {
|
extends Group {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -65,7 +65,7 @@ final class RoundRobinRoutingLogic extends RoutingLogic {
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
final case class RoundRobinPool(
|
final case class RoundRobinPool(
|
||||||
override val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
|
val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
|
||||||
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
||||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||||
override val usePoolDispatcher: Boolean = false)
|
override val usePoolDispatcher: Boolean = false)
|
||||||
|
|
@ -128,7 +128,7 @@ final case class RoundRobinPool(
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
final case class RoundRobinGroup(
|
final case class RoundRobinGroup(
|
||||||
override val paths: immutable.Iterable[String],
|
val paths: immutable.Iterable[String],
|
||||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
||||||
extends Group {
|
extends Group {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -102,17 +102,11 @@ private[akka] class RoutedActorCell(
|
||||||
_router = routerConfig.createRouter(system)
|
_router = routerConfig.createRouter(system)
|
||||||
routerConfig match {
|
routerConfig match {
|
||||||
case pool: Pool ⇒
|
case pool: Pool ⇒
|
||||||
// must not use pool.nrOfInstances(system) for old (not re-compiled) custom routers
|
val nrOfRoutees = pool.nrOfInstances(system)
|
||||||
// for binary backwards compatibility reasons
|
|
||||||
val deprecatedNrOfInstances = pool.nrOfInstances
|
|
||||||
val nrOfRoutees = if (deprecatedNrOfInstances < 0) pool.nrOfInstances(system) else deprecatedNrOfInstances
|
|
||||||
if (nrOfRoutees > 0)
|
if (nrOfRoutees > 0)
|
||||||
addRoutees(Vector.fill(nrOfRoutees)(pool.newRoutee(routeeProps, this)))
|
addRoutees(Vector.fill(nrOfRoutees)(pool.newRoutee(routeeProps, this)))
|
||||||
case group: Group ⇒
|
case group: Group ⇒
|
||||||
// must not use group.paths(system) for old (not re-compiled) custom routers
|
val paths = group.paths(system)
|
||||||
// for binary backwards compatibility reasons
|
|
||||||
val deprecatedPaths = group.paths
|
|
||||||
val paths = if (deprecatedPaths == null) group.paths(system) else deprecatedPaths
|
|
||||||
if (paths.nonEmpty)
|
if (paths.nonEmpty)
|
||||||
addRoutees(paths.map(p ⇒ group.routeeFor(p, this))(collection.breakOut))
|
addRoutees(paths.map(p ⇒ group.routeeFor(p, this))(collection.breakOut))
|
||||||
case _ ⇒
|
case _ ⇒
|
||||||
|
|
|
||||||
|
|
@ -128,15 +128,6 @@ private[akka] trait PoolOverrideUnsetConfig[T <: Pool] extends Pool {
|
||||||
* Java API: Base class for custom router [[Group]]
|
* Java API: Base class for custom router [[Group]]
|
||||||
*/
|
*/
|
||||||
abstract class GroupBase extends Group {
|
abstract class GroupBase extends Group {
|
||||||
@deprecated("Implement getPaths with ActorSystem parameter instead", "2.4")
|
|
||||||
def getPaths: java.lang.Iterable[String] = null
|
|
||||||
|
|
||||||
@deprecated("Use paths with ActorSystem parameter instead", "2.4")
|
|
||||||
override final def paths: immutable.Iterable[String] = {
|
|
||||||
val tmp = getPaths
|
|
||||||
if (tmp != null) immutableSeq(tmp)
|
|
||||||
else null
|
|
||||||
}
|
|
||||||
|
|
||||||
def getPaths(system: ActorSystem): java.lang.Iterable[String]
|
def getPaths(system: ActorSystem): java.lang.Iterable[String]
|
||||||
|
|
||||||
|
|
@ -151,9 +142,6 @@ abstract class GroupBase extends Group {
|
||||||
*/
|
*/
|
||||||
trait Group extends RouterConfig {
|
trait Group extends RouterConfig {
|
||||||
|
|
||||||
@deprecated("Implement paths with ActorSystem parameter instead", "2.4")
|
|
||||||
def paths: immutable.Iterable[String] = null
|
|
||||||
|
|
||||||
def paths(system: ActorSystem): immutable.Iterable[String]
|
def paths(system: ActorSystem): immutable.Iterable[String]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -191,9 +179,6 @@ abstract class PoolBase extends Pool
|
||||||
*/
|
*/
|
||||||
trait Pool extends RouterConfig {
|
trait Pool extends RouterConfig {
|
||||||
|
|
||||||
@deprecated("Implement nrOfInstances with ActorSystem parameter instead", "2.4")
|
|
||||||
def nrOfInstances: Int = -1
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Initial number of routee instances
|
* Initial number of routee instances
|
||||||
*/
|
*/
|
||||||
|
|
|
||||||
|
|
@ -96,7 +96,7 @@ private[akka] final case class ScatterGatherFirstCompletedRoutees(
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
final case class ScatterGatherFirstCompletedPool(
|
final case class ScatterGatherFirstCompletedPool(
|
||||||
override val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
|
val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
|
||||||
within: FiniteDuration,
|
within: FiniteDuration,
|
||||||
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
||||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||||
|
|
@ -165,7 +165,7 @@ final case class ScatterGatherFirstCompletedPool(
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
final case class ScatterGatherFirstCompletedGroup(
|
final case class ScatterGatherFirstCompletedGroup(
|
||||||
override val paths: immutable.Iterable[String],
|
val paths: immutable.Iterable[String],
|
||||||
within: FiniteDuration,
|
within: FiniteDuration,
|
||||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
||||||
extends Group {
|
extends Group {
|
||||||
|
|
|
||||||
|
|
@ -173,7 +173,7 @@ class SmallestMailboxRoutingLogic extends RoutingLogic {
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
final case class SmallestMailboxPool(
|
final case class SmallestMailboxPool(
|
||||||
override val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
|
val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
|
||||||
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
||||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||||
override val usePoolDispatcher: Boolean = false)
|
override val usePoolDispatcher: Boolean = false)
|
||||||
|
|
|
||||||
|
|
@ -141,7 +141,7 @@ private[akka] final case class TailChoppingRoutees(
|
||||||
*/
|
*/
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
final case class TailChoppingPool(
|
final case class TailChoppingPool(
|
||||||
override val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
|
val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
|
||||||
within: FiniteDuration,
|
within: FiniteDuration,
|
||||||
interval: FiniteDuration,
|
interval: FiniteDuration,
|
||||||
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
||||||
|
|
@ -227,7 +227,7 @@ final case class TailChoppingPool(
|
||||||
* router management messages
|
* router management messages
|
||||||
*/
|
*/
|
||||||
final case class TailChoppingGroup(
|
final case class TailChoppingGroup(
|
||||||
override val paths: immutable.Iterable[String],
|
val paths: immutable.Iterable[String],
|
||||||
within: FiniteDuration,
|
within: FiniteDuration,
|
||||||
interval: FiniteDuration,
|
interval: FiniteDuration,
|
||||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId) extends Group {
|
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId) extends Group {
|
||||||
|
|
|
||||||
|
|
@ -281,14 +281,6 @@ object JavaSerializer {
|
||||||
*/
|
*/
|
||||||
class JavaSerializer(val system: ExtendedActorSystem) extends BaseSerializer {
|
class JavaSerializer(val system: ExtendedActorSystem) extends BaseSerializer {
|
||||||
|
|
||||||
@deprecated("Use constructor with ExtendedActorSystem", "2.4")
|
|
||||||
def this() = this(null)
|
|
||||||
|
|
||||||
// TODO remove this when deprecated this() is removed
|
|
||||||
override val identifier: Int =
|
|
||||||
if (system eq null) 1
|
|
||||||
else identifierFromConfig
|
|
||||||
|
|
||||||
def includeManifest: Boolean = false
|
def includeManifest: Boolean = false
|
||||||
|
|
||||||
def toBinary(o: AnyRef): Array[Byte] = {
|
def toBinary(o: AnyRef): Array[Byte] = {
|
||||||
|
|
@ -324,14 +316,6 @@ class NullSerializer extends Serializer {
|
||||||
*/
|
*/
|
||||||
class ByteArraySerializer(val system: ExtendedActorSystem) extends BaseSerializer with ByteBufferSerializer {
|
class ByteArraySerializer(val system: ExtendedActorSystem) extends BaseSerializer with ByteBufferSerializer {
|
||||||
|
|
||||||
@deprecated("Use constructor with ExtendedActorSystem", "2.4")
|
|
||||||
def this() = this(null)
|
|
||||||
|
|
||||||
// TODO remove this when deprecated this() is removed
|
|
||||||
override val identifier: Int =
|
|
||||||
if (system eq null) 4
|
|
||||||
else identifierFromConfig
|
|
||||||
|
|
||||||
def includeManifest: Boolean = false
|
def includeManifest: Boolean = false
|
||||||
def toBinary(o: AnyRef): Array[Byte] = o match {
|
def toBinary(o: AnyRef): Array[Byte] = o match {
|
||||||
case null ⇒ null
|
case null ⇒ null
|
||||||
|
|
|
||||||
|
|
@ -1,41 +0,0 @@
|
||||||
/**
|
|
||||||
* Copyright (C) 2009-2017 Lightbend Inc. <http://www.lightbend.com>
|
|
||||||
*/
|
|
||||||
|
|
||||||
package akka.util
|
|
||||||
|
|
||||||
import java.security.{ MessageDigest, SecureRandom }
|
|
||||||
@deprecated(message = "Not in use.", since = "2.4-M1")
|
|
||||||
object Crypt {
|
|
||||||
val hex = "0123456789ABCDEF"
|
|
||||||
val lineSeparator = System.getProperty("line.separator")
|
|
||||||
|
|
||||||
lazy val random = SecureRandom.getInstance("SHA1PRNG")
|
|
||||||
|
|
||||||
def md5(text: String): String = md5(unifyLineSeparator(text).getBytes("ASCII"))
|
|
||||||
|
|
||||||
def md5(bytes: Array[Byte]): String = digest(bytes, MessageDigest.getInstance("MD5"))
|
|
||||||
|
|
||||||
def sha1(text: String): String = sha1(unifyLineSeparator(text).getBytes("ASCII"))
|
|
||||||
|
|
||||||
def sha1(bytes: Array[Byte]): String = digest(bytes, MessageDigest.getInstance("SHA1"))
|
|
||||||
|
|
||||||
def generateSecureCookie: String = {
|
|
||||||
val bytes = Array.fill(32)(0.byteValue)
|
|
||||||
random.nextBytes(bytes)
|
|
||||||
sha1(bytes)
|
|
||||||
}
|
|
||||||
|
|
||||||
def digest(bytes: Array[Byte], md: MessageDigest): String = {
|
|
||||||
md.update(bytes)
|
|
||||||
hexify(md.digest)
|
|
||||||
}
|
|
||||||
|
|
||||||
def hexify(bytes: Array[Byte]): String = {
|
|
||||||
val builder = new java.lang.StringBuilder(bytes.length * 2)
|
|
||||||
bytes.foreach { byte ⇒ builder.append(hex.charAt((byte & 0xF0) >> 4)).append(hex.charAt(byte & 0xF)) }
|
|
||||||
builder.toString
|
|
||||||
}
|
|
||||||
|
|
||||||
private def unifyLineSeparator(text: String): String = text.replaceAll(lineSeparator, "\n")
|
|
||||||
}
|
|
||||||
|
|
@ -121,7 +121,7 @@ final case class AdaptiveLoadBalancingRoutingLogic(system: ActorSystem, metricsS
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
final case class AdaptiveLoadBalancingPool(
|
final case class AdaptiveLoadBalancingPool(
|
||||||
metricsSelector: MetricsSelector = MixMetricsSelector,
|
metricsSelector: MetricsSelector = MixMetricsSelector,
|
||||||
override val nrOfInstances: Int = 0,
|
val nrOfInstances: Int = 0,
|
||||||
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
||||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||||
override val usePoolDispatcher: Boolean = false)
|
override val usePoolDispatcher: Boolean = false)
|
||||||
|
|
@ -203,7 +203,7 @@ final case class AdaptiveLoadBalancingPool(
|
||||||
@SerialVersionUID(1L)
|
@SerialVersionUID(1L)
|
||||||
final case class AdaptiveLoadBalancingGroup(
|
final case class AdaptiveLoadBalancingGroup(
|
||||||
metricsSelector: MetricsSelector = MixMetricsSelector,
|
metricsSelector: MetricsSelector = MixMetricsSelector,
|
||||||
override val paths: immutable.Iterable[String] = Nil,
|
val paths: immutable.Iterable[String] = Nil,
|
||||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
||||||
extends Group {
|
extends Group {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -26,14 +26,6 @@ import akka.cluster.InternalClusterAction.ExitingConfirmed
|
||||||
*/
|
*/
|
||||||
class ClusterMessageSerializer(val system: ExtendedActorSystem) extends BaseSerializer {
|
class ClusterMessageSerializer(val system: ExtendedActorSystem) extends BaseSerializer {
|
||||||
|
|
||||||
@deprecated("Use constructor with ExtendedActorSystem", "2.4")
|
|
||||||
def this() = this(null)
|
|
||||||
|
|
||||||
// TODO remove this when deprecated this() is removed
|
|
||||||
override val identifier: Int =
|
|
||||||
if (system eq null) 5
|
|
||||||
else identifierFromConfig
|
|
||||||
|
|
||||||
private final val BufferSize = 1024 * 4
|
private final val BufferSize = 1024 * 4
|
||||||
// must be lazy because serializer is initialized from Cluster extension constructor
|
// must be lazy because serializer is initialized from Cluster extension constructor
|
||||||
private lazy val GossipTimeToLive = Cluster(system).settings.GossipTimeToLive
|
private lazy val GossipTimeToLive = Cluster(system).settings.GossipTimeToLive
|
||||||
|
|
|
||||||
|
|
@ -344,10 +344,6 @@ private[persistence] trait Eventsourced extends Snapshotter with PersistenceStas
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@deprecated("use persistAll instead", "2.4")
|
|
||||||
def persist[A](events: immutable.Seq[A])(handler: A ⇒ Unit): Unit =
|
|
||||||
persistAll(events)(handler)
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Asynchronously persists `event`. On successful persistence, `handler` is called with the
|
* Asynchronously persists `event`. On successful persistence, `handler` is called with the
|
||||||
* persisted event.
|
* persisted event.
|
||||||
|
|
@ -397,10 +393,6 @@ private[persistence] trait Eventsourced extends Snapshotter with PersistenceStas
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@deprecated("use persistAllAsync instead", "2.4")
|
|
||||||
def persistAsync[A](events: immutable.Seq[A])(handler: A ⇒ Unit): Unit =
|
|
||||||
persistAllAsync(events)(handler)
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Defer the handler execution until all pending handlers have been executed.
|
* Defer the handler execution until all pending handlers have been executed.
|
||||||
* Allows to define logic within the actor, which will respect the invocation-order-guarantee
|
* Allows to define logic within the actor, which will respect the invocation-order-guarantee
|
||||||
|
|
|
||||||
|
|
@ -217,10 +217,6 @@ abstract class UntypedPersistentActor extends UntypedActor with Eventsourced wit
|
||||||
def persistAll[A](events: JIterable[A], handler: Procedure[A]): Unit =
|
def persistAll[A](events: JIterable[A], handler: Procedure[A]): Unit =
|
||||||
persistAll(Util.immutableSeq(events))(event ⇒ handler(event))
|
persistAll(Util.immutableSeq(events))(event ⇒ handler(event))
|
||||||
|
|
||||||
@deprecated("use persistAll instead", "2.4")
|
|
||||||
def persist[A](events: JIterable[A], handler: Procedure[A]): Unit =
|
|
||||||
persistAll(events, handler)
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* JAVA API: asynchronously persists `event`. On successful persistence, `handler` is called with the
|
* JAVA API: asynchronously persists `event`. On successful persistence, `handler` is called with the
|
||||||
* persisted event.
|
* persisted event.
|
||||||
|
|
@ -377,10 +373,6 @@ abstract class AbstractPersistentActor extends AbstractActor with PersistentActo
|
||||||
def persistAll[A](events: JIterable[A], handler: Procedure[A]): Unit =
|
def persistAll[A](events: JIterable[A], handler: Procedure[A]): Unit =
|
||||||
persistAll(Util.immutableSeq(events))(event ⇒ handler(event))
|
persistAll(Util.immutableSeq(events))(event ⇒ handler(event))
|
||||||
|
|
||||||
@deprecated("use persistAll instead", "2.4")
|
|
||||||
def persist[A](events: JIterable[A], handler: Procedure[A]): Unit =
|
|
||||||
persistAll(events, handler)
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Java API: asynchronously persists `event`. On successful persistence, `handler` is called with the
|
* Java API: asynchronously persists `event`. On successful persistence, `handler` is called with the
|
||||||
* persisted event.
|
* persisted event.
|
||||||
|
|
@ -413,10 +405,6 @@ abstract class AbstractPersistentActor extends AbstractActor with PersistentActo
|
||||||
def persistAllAsync[A](events: JIterable[A], handler: Procedure[A]): Unit =
|
def persistAllAsync[A](events: JIterable[A], handler: Procedure[A]): Unit =
|
||||||
persistAllAsync(Util.immutableSeq(events))(event ⇒ handler(event))
|
persistAllAsync(Util.immutableSeq(events))(event ⇒ handler(event))
|
||||||
|
|
||||||
@deprecated("use persistAllAsync instead", "2.4")
|
|
||||||
def persistAsync[A](events: JIterable[A], handler: Procedure[A]): Unit =
|
|
||||||
persistAllAsync(events, handler)
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Defer the handler execution until all pending handlers have been executed.
|
* Defer the handler execution until all pending handlers have been executed.
|
||||||
* Allows to define logic within the actor, which will respect the invocation-order-guarantee
|
* Allows to define logic within the actor, which will respect the invocation-order-guarantee
|
||||||
|
|
|
||||||
|
|
@ -41,11 +41,6 @@ class DeadlineFailureDetector(
|
||||||
acceptableHeartbeatPause = config.getMillisDuration("acceptable-heartbeat-pause"),
|
acceptableHeartbeatPause = config.getMillisDuration("acceptable-heartbeat-pause"),
|
||||||
heartbeatInterval = config.getMillisDuration("heartbeat-interval"))
|
heartbeatInterval = config.getMillisDuration("heartbeat-interval"))
|
||||||
|
|
||||||
// for backwards compatibility with 2.3.x
|
|
||||||
@deprecated("Use constructor with acceptableHeartbeatPause and heartbeatInterval", "2.4")
|
|
||||||
def this(acceptableHeartbeatPause: FiniteDuration)(implicit clock: Clock) =
|
|
||||||
this(acceptableHeartbeatPause, heartbeatInterval = 1.millis)(clock)
|
|
||||||
|
|
||||||
require(acceptableHeartbeatPause >= Duration.Zero, "failure-detector.acceptable-heartbeat-pause must be >= 0 s")
|
require(acceptableHeartbeatPause >= Duration.Zero, "failure-detector.acceptable-heartbeat-pause must be >= 0 s")
|
||||||
require(heartbeatInterval > Duration.Zero, "failure-detector.heartbeat-interval must be > 0 s")
|
require(heartbeatInterval > Duration.Zero, "failure-detector.heartbeat-interval must be > 0 s")
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -84,7 +84,7 @@ final case class RemotingErrorEvent(cause: Throwable) extends RemotingLifecycleE
|
||||||
// For binary compatibility
|
// For binary compatibility
|
||||||
object QuarantinedEvent extends AbstractFunction2[Address, Int, QuarantinedEvent] {
|
object QuarantinedEvent extends AbstractFunction2[Address, Int, QuarantinedEvent] {
|
||||||
|
|
||||||
@deprecated("Use long uid apply")
|
@deprecated("Use long uid apply", "2.4.x")
|
||||||
def apply(address: Address, uid: Int) = new QuarantinedEvent(address, uid)
|
def apply(address: Address, uid: Int) = new QuarantinedEvent(address, uid)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -99,13 +99,13 @@ final case class QuarantinedEvent(address: Address, longUid: Long) extends Remot
|
||||||
|
|
||||||
// For binary compatibility
|
// For binary compatibility
|
||||||
|
|
||||||
@deprecated("Use long uid constructor")
|
@deprecated("Use long uid constructor", "2.4.x")
|
||||||
def this(address: Address, uid: Int) = this(address, uid.toLong)
|
def this(address: Address, uid: Int) = this(address, uid.toLong)
|
||||||
|
|
||||||
@deprecated("Use long uid")
|
@deprecated("Use long uid", "2.4.x")
|
||||||
def uid: Int = longUid.toInt
|
def uid: Int = longUid.toInt
|
||||||
|
|
||||||
@deprecated("Use long uid copy method")
|
@deprecated("Use long uid copy method", "2.4.x")
|
||||||
def copy(address: Address = address, uid: Int = uid) = new QuarantinedEvent(address, uid)
|
def copy(address: Address = address, uid: Int = uid) = new QuarantinedEvent(address, uid)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -18,16 +18,8 @@ import akka.serialization.SerializerWithStringManifest
|
||||||
|
|
||||||
class MessageContainerSerializer(val system: ExtendedActorSystem) extends BaseSerializer {
|
class MessageContainerSerializer(val system: ExtendedActorSystem) extends BaseSerializer {
|
||||||
|
|
||||||
@deprecated("Use constructor with ExtendedActorSystem", "2.4")
|
|
||||||
def this() = this(null)
|
|
||||||
|
|
||||||
private lazy val serialization = SerializationExtension(system)
|
private lazy val serialization = SerializationExtension(system)
|
||||||
|
|
||||||
// TODO remove this when deprecated this() is removed
|
|
||||||
override val identifier: Int =
|
|
||||||
if (system eq null) 6
|
|
||||||
else identifierFromConfig
|
|
||||||
|
|
||||||
def includeManifest: Boolean = false
|
def includeManifest: Boolean = false
|
||||||
|
|
||||||
def toBinary(obj: AnyRef): Array[Byte] = obj match {
|
def toBinary(obj: AnyRef): Array[Byte] = obj match {
|
||||||
|
|
|
||||||
|
|
@ -40,17 +40,6 @@ object ProtobufSerializer {
|
||||||
*/
|
*/
|
||||||
class ProtobufSerializer(val system: ExtendedActorSystem) extends BaseSerializer {
|
class ProtobufSerializer(val system: ExtendedActorSystem) extends BaseSerializer {
|
||||||
|
|
||||||
@deprecated("Use constructor with ExtendedActorSystem", "2.4")
|
|
||||||
def this() = this(null)
|
|
||||||
|
|
||||||
// TODO remove this when deprecated this() is removed
|
|
||||||
override val identifier: Int =
|
|
||||||
if (system eq null) 2
|
|
||||||
else identifierFromConfig
|
|
||||||
|
|
||||||
@deprecated("Will be removed without replacement", "2.4")
|
|
||||||
val ARRAY_OF_BYTE_ARRAY = Array[Class[_]](classOf[Array[Byte]])
|
|
||||||
|
|
||||||
private val parsingMethodBindingRef = new AtomicReference[Map[Class[_], Method]](Map.empty)
|
private val parsingMethodBindingRef = new AtomicReference[Map[Class[_], Method]](Map.empty)
|
||||||
private val toByteArrayMethodBindingRef = new AtomicReference[Map[Class[_], Method]](Map.empty)
|
private val toByteArrayMethodBindingRef = new AtomicReference[Map[Class[_], Method]](Map.empty)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -13,6 +13,7 @@ import akka.event.Logging
|
||||||
import akka.testkit.TestEvent
|
import akka.testkit.TestEvent
|
||||||
import akka.testkit.EventFilter
|
import akka.testkit.EventFilter
|
||||||
import org.testng.annotations.BeforeClass
|
import org.testng.annotations.BeforeClass
|
||||||
|
import scala.concurrent.Await
|
||||||
|
|
||||||
trait ActorSystemLifecycle {
|
trait ActorSystemLifecycle {
|
||||||
|
|
||||||
|
|
@ -31,8 +32,7 @@ trait ActorSystemLifecycle {
|
||||||
@AfterClass
|
@AfterClass
|
||||||
def shutdownActorSystem(): Unit = {
|
def shutdownActorSystem(): Unit = {
|
||||||
try {
|
try {
|
||||||
system.terminate()
|
Await.ready(system.terminate(), shutdownTimeout)
|
||||||
system.awaitTermination(shutdownTimeout)
|
|
||||||
} catch {
|
} catch {
|
||||||
case _: TimeoutException ⇒
|
case _: TimeoutException ⇒
|
||||||
val msg = "Failed to stop [%s] within [%s] \n%s".format(system.name, shutdownTimeout,
|
val msg = "Failed to stop [%s] within [%s] \n%s".format(system.name, shutdownTimeout,
|
||||||
|
|
|
||||||
|
|
@ -413,7 +413,9 @@ class TcpSpec extends StreamSpec("akka.stream.materializer.subscription-timeout.
|
||||||
|
|
||||||
result.failed.futureValue shouldBe a[StreamTcpException]
|
result.failed.futureValue shouldBe a[StreamTcpException]
|
||||||
|
|
||||||
binding.map(_.unbind()).recover { case NonFatal(_) ⇒ () } foreach (_ ⇒ system2.shutdown())
|
binding.map(_.unbind()).recover { case NonFatal(_) ⇒ () }.foreach { _ ⇒
|
||||||
|
shutdown(system2)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -258,6 +258,80 @@ object MiMa extends AutoPlugin {
|
||||||
// object akka.stream.stage.StatefulStage#Finish does not have a correspondent in current version
|
// object akka.stream.stage.StatefulStage#Finish does not have a correspondent in current version
|
||||||
ProblemFilters.exclude[MissingClassProblem]("akka.stream.stage.StatefulStage$Finish$"),
|
ProblemFilters.exclude[MissingClassProblem]("akka.stream.stage.StatefulStage$Finish$"),
|
||||||
|
|
||||||
|
// #21423 remove deprecated ActorSystem termination methods (in 2.5.x)
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.ActorSystemImpl.shutdown"),
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.ActorSystemImpl.isTerminated"),
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.ActorSystemImpl.awaitTermination"),
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.ActorSystemImpl.awaitTermination"),
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.ActorSystem.shutdown"),
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.ActorSystem.isTerminated"),
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.ActorSystem.awaitTermination"),
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.ActorSystem.awaitTermination"),
|
||||||
|
|
||||||
|
// #21423 remove deprecated ActorPath.ElementRegex
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.actor.ActorPath.ElementRegex"),
|
||||||
|
|
||||||
|
// #21423 remove some deprecated event bus classes
|
||||||
|
ProblemFilters.exclude[MissingClassProblem]("akka.event.ActorClassification"),
|
||||||
|
ProblemFilters.exclude[MissingClassProblem]("akka.event.EventStream$"),
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.event.EventStream.this"),
|
||||||
|
ProblemFilters.exclude[MissingClassProblem]("akka.event.japi.ActorEventBus"),
|
||||||
|
|
||||||
|
// #21423 remove deprecated util.Crypt
|
||||||
|
ProblemFilters.exclude[MissingClassProblem]("akka.util.Crypt"),
|
||||||
|
ProblemFilters.exclude[MissingClassProblem]("akka.util.Crypt$"),
|
||||||
|
|
||||||
|
// #21423 removal of deprecated serializer constructors (in 2.5.x)
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.serialization.ProtobufSerializer.this"),
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.serialization.MessageContainerSerializer.this"),
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.serialization.DaemonMsgCreateSerializer.this"),
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.serialization.JavaSerializer.this"),
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.serialization.ByteArraySerializer.this"),
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.protobuf.ClusterMessageSerializer.this"),
|
||||||
|
|
||||||
|
// #21423 removal of deprecated constructor in PromiseActorRef
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.pattern.PromiseActorRef.this"),
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.pattern.PromiseActorRef.apply"),
|
||||||
|
|
||||||
|
// #21423 remove deprecated methods in routing
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.routing.Pool.nrOfInstances"),
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.routing.Group.paths"),
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.routing.PoolBase.nrOfInstances"),
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.routing.GroupBase.paths"),
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.routing.GroupBase.getPaths"),
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.routing.FromConfig.nrOfInstances"),
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.routing.RemoteRouterConfig.nrOfInstances"),
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.routing.ClusterRouterGroup.paths"),
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.cluster.routing.ClusterRouterPool.nrOfInstances"),
|
||||||
|
|
||||||
|
// #21423 remove deprecated persist method (persistAll)
|
||||||
|
// This might filter changes to the ordinary persist method also, but not much to do about that
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.persistence.UntypedPersistentActor.persist"),
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.persistence.UntypedPersistentActor.persist"),
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.persistence.UntypedPersistentActor.persistAsync"),
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.persistence.Eventsourced.persist"),
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.persistence.Eventsourced.persistAsync"),
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.persistence.AbstractPersistentActor.persist"),
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.persistence.AbstractPersistentActor.persist"),
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.persistence.AbstractPersistentActor.persistAsync"),
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.persistence.AbstractPersistentActor.persistAsync"),
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.persistence.fsm.AbstractPersistentFSM.persist"),
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.persistence.fsm.AbstractPersistentFSM.persistAsync"),
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.persistence.fsm.AbstractPersistentLoggingFSM.persist"),
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.persistence.fsm.AbstractPersistentLoggingFSM.persistAsync"),
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.cluster.sharding.PersistentShard.persist"),
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.cluster.sharding.PersistentShard.persistAsync"),
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.cluster.sharding.PersistentShardCoordinator.persist"),
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.cluster.sharding.PersistentShardCoordinator.persistAsync"),
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.cluster.sharding.RemoveInternalClusterShardingData#RemoveOnePersistenceId.persist"),
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.cluster.sharding.RemoveInternalClusterShardingData#RemoveOnePersistenceId.persistAsync"),
|
||||||
|
|
||||||
|
// #21423 remove deprecated ARRAY_OF_BYTE_ARRAY
|
||||||
|
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.remote.serialization.ProtobufSerializer.ARRAY_OF_BYTE_ARRAY"),
|
||||||
|
|
||||||
|
// #21423 remove deprecated constructor in DeadlineFailureDetector
|
||||||
|
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.remote.DeadlineFailureDetector.this"),
|
||||||
|
|
||||||
// #21423 removal of deprecated `PersistentView` (in 2.5.x)
|
// #21423 removal of deprecated `PersistentView` (in 2.5.x)
|
||||||
ProblemFilters.exclude[MissingClassProblem]("akka.persistence.Update"),
|
ProblemFilters.exclude[MissingClassProblem]("akka.persistence.Update"),
|
||||||
ProblemFilters.exclude[MissingClassProblem]("akka.persistence.Update$"),
|
ProblemFilters.exclude[MissingClassProblem]("akka.persistence.Update$"),
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue