Merge pull request #22206 from akka/wip-21423-remove-deprecations-patriknw

remove deprecations, #21423
This commit is contained in:
Patrik Nordwall 2017-01-24 16:45:31 +01:00 committed by GitHub
commit 94e40460a4
32 changed files with 105 additions and 361 deletions

View file

@ -197,30 +197,26 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend
import system.dispatcher
system2.scheduler.scheduleOnce(200.millis.dilated) { system2.terminate() }
system2.awaitTermination(5 seconds)
Await.ready(system2.whenTerminated, 5 seconds)
callbackWasRun should ===(true)
}
"return isTerminated status correctly" in {
val system = ActorSystem().asInstanceOf[ActorSystemImpl]
system.isTerminated should ===(false)
val wt = system.whenTerminated
wt.isCompleted should ===(false)
val f = system.terminate()
val terminated = Await.result(wt, 10 seconds)
system.whenTerminated.isCompleted should ===(true)
terminated.actor should ===(system.provider.rootGuardian)
terminated.addressTerminated should ===(true)
terminated.existenceConfirmed should ===(true)
terminated should be theSameInstanceAs Await.result(f, 10 seconds)
system.awaitTermination(10 seconds)
system.isTerminated should ===(true)
}
"throw RejectedExecutionException when shutdown" in {
val system2 = ActorSystem("AwaitTermination", AkkaSpec.testConf)
Await.ready(system2.terminate(), 10 seconds)
system2.awaitTermination(10 seconds)
intercept[RejectedExecutionException] {
system2.registerOnTermination { println("IF YOU SEE THIS THEN THERE'S A BUG HERE") }

View file

@ -44,7 +44,7 @@ class AddressTerminatedTopicBenchSpec extends AkkaSpec("akka.loglevel=INFO") {
shutdown(sys, 10.seconds, verifySystemShutdown = true)
log.info("Stopping {} actors took {} ms", num, (System.nanoTime() - t2).nanos.toMillis)
} finally {
if (!sys.isTerminated) shutdown(sys)
shutdown(sys)
}
}

View file

@ -58,9 +58,6 @@ object ActorPath {
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 ValidPathCode = -1

View file

@ -3,6 +3,7 @@
*/
package akka.actor
import scala.language.implicitConversions
import java.util.concurrent.CompletionStage
import scala.annotation.tailrec
@ -231,7 +232,7 @@ object ActorSelection {
matchingChildren.foreach(_.tell(sel.msg, sender))
} else {
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)
emptyRef.tell(sel, sender)
else {

View file

@ -523,42 +523,6 @@ abstract class ActorSystem extends ActorRefFactory {
*/
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
* 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 registerOnTermination[T](code: T) { registerOnTermination(new Runnable { def run = 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] = {
if (!settings.LogDeadLettersDuringShutdown) logDeadLetterListener foreach stop

View file

@ -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)
}

View file

@ -11,11 +11,6 @@ import akka.util.Subclassification
import java.util.concurrent.atomic.AtomicReference
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,
* 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)
@deprecated("Use constructor with ActorSystem parameter", "2.4")
def this(debug: Boolean = false) = this(sys = null, debug)
type Event = AnyRef
type Classifier = Class[_]

View file

@ -217,35 +217,3 @@ abstract class ManagedActorEventBus[E](system: ActorSystem) extends EventBus[E,
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)
}

View file

@ -438,9 +438,6 @@ private[akka] final class PromiseActorRef private (val provider: ActorRefProvide
import AbstractPromiseActorRef.{ stateOffset, watchedByOffset }
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.
@volatile var messageClassName = _mcn
@ -607,7 +604,4 @@ private[akka] object PromiseActorRef {
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)
}

View file

@ -66,7 +66,7 @@ private[akka] final class BalancingRoutingLogic extends RoutingLogic {
*/
@SerialVersionUID(1L)
final case class BalancingPool(
override val nrOfInstances: Int,
val nrOfInstances: Int,
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
extends Pool {

View file

@ -56,7 +56,7 @@ final class BroadcastRoutingLogic extends RoutingLogic {
*/
@SerialVersionUID(1L)
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 routerDispatcher: String = Dispatchers.DefaultDispatcherId,
override val usePoolDispatcher: Boolean = false)
@ -118,7 +118,7 @@ final case class BroadcastPool(
*/
@SerialVersionUID(1L)
final case class BroadcastGroup(
override val paths: immutable.Iterable[String],
val paths: immutable.Iterable[String],
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
extends Group {

View file

@ -267,7 +267,7 @@ final case class ConsistentHashingRoutingLogic(
*/
@SerialVersionUID(1L)
final case class ConsistentHashingPool(
override val nrOfInstances: Int,
val nrOfInstances: Int,
override val resizer: Option[Resizer] = None,
val virtualNodesFactor: Int = 0,
val hashMapping: ConsistentHashingRouter.ConsistentHashMapping = ConsistentHashingRouter.emptyConsistentHashMapping,
@ -355,7 +355,7 @@ final case class ConsistentHashingPool(
*/
@SerialVersionUID(1L)
final case class ConsistentHashingGroup(
override val paths: immutable.Iterable[String],
val paths: immutable.Iterable[String],
val virtualNodesFactor: Int = 0,
val hashMapping: ConsistentHashingRouter.ConsistentHashMapping = ConsistentHashingRouter.emptyConsistentHashMapping,
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)

View file

@ -57,7 +57,7 @@ final class RandomRoutingLogic extends RoutingLogic {
*/
@SerialVersionUID(1L)
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 routerDispatcher: String = Dispatchers.DefaultDispatcherId,
override val usePoolDispatcher: Boolean = false)
@ -119,7 +119,7 @@ final case class RandomPool(
*/
@SerialVersionUID(1L)
final case class RandomGroup(
override val paths: immutable.Iterable[String],
val paths: immutable.Iterable[String],
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
extends Group {

View file

@ -65,7 +65,7 @@ final class RoundRobinRoutingLogic extends RoutingLogic {
*/
@SerialVersionUID(1L)
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 routerDispatcher: String = Dispatchers.DefaultDispatcherId,
override val usePoolDispatcher: Boolean = false)
@ -128,7 +128,7 @@ final case class RoundRobinPool(
*/
@SerialVersionUID(1L)
final case class RoundRobinGroup(
override val paths: immutable.Iterable[String],
val paths: immutable.Iterable[String],
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
extends Group {

View file

@ -102,17 +102,11 @@ private[akka] class RoutedActorCell(
_router = routerConfig.createRouter(system)
routerConfig match {
case pool: Pool
// must not use pool.nrOfInstances(system) for old (not re-compiled) custom routers
// for binary backwards compatibility reasons
val deprecatedNrOfInstances = pool.nrOfInstances
val nrOfRoutees = if (deprecatedNrOfInstances < 0) pool.nrOfInstances(system) else deprecatedNrOfInstances
val nrOfRoutees = pool.nrOfInstances(system)
if (nrOfRoutees > 0)
addRoutees(Vector.fill(nrOfRoutees)(pool.newRoutee(routeeProps, this)))
case group: Group
// must not use group.paths(system) for old (not re-compiled) custom routers
// for binary backwards compatibility reasons
val deprecatedPaths = group.paths
val paths = if (deprecatedPaths == null) group.paths(system) else deprecatedPaths
val paths = group.paths(system)
if (paths.nonEmpty)
addRoutees(paths.map(p group.routeeFor(p, this))(collection.breakOut))
case _

View file

@ -128,15 +128,6 @@ private[akka] trait PoolOverrideUnsetConfig[T <: Pool] extends Pool {
* Java API: Base class for custom router [[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]
@ -151,9 +142,6 @@ abstract class GroupBase extends Group {
*/
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]
/**
@ -191,9 +179,6 @@ abstract class PoolBase extends Pool
*/
trait Pool extends RouterConfig {
@deprecated("Implement nrOfInstances with ActorSystem parameter instead", "2.4")
def nrOfInstances: Int = -1
/**
* Initial number of routee instances
*/

View file

@ -96,7 +96,7 @@ private[akka] final case class ScatterGatherFirstCompletedRoutees(
*/
@SerialVersionUID(1L)
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,
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
@ -165,7 +165,7 @@ final case class ScatterGatherFirstCompletedPool(
*/
@SerialVersionUID(1L)
final case class ScatterGatherFirstCompletedGroup(
override val paths: immutable.Iterable[String],
val paths: immutable.Iterable[String],
within: FiniteDuration,
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
extends Group {

View file

@ -173,7 +173,7 @@ class SmallestMailboxRoutingLogic extends RoutingLogic {
*/
@SerialVersionUID(1L)
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 routerDispatcher: String = Dispatchers.DefaultDispatcherId,
override val usePoolDispatcher: Boolean = false)

View file

@ -141,7 +141,7 @@ private[akka] final case class TailChoppingRoutees(
*/
@SerialVersionUID(1L)
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,
interval: FiniteDuration,
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
@ -227,7 +227,7 @@ final case class TailChoppingPool(
* router management messages
*/
final case class TailChoppingGroup(
override val paths: immutable.Iterable[String],
val paths: immutable.Iterable[String],
within: FiniteDuration,
interval: FiniteDuration,
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId) extends Group {

View file

@ -281,14 +281,6 @@ object JavaSerializer {
*/
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 toBinary(o: AnyRef): Array[Byte] = {
@ -324,14 +316,6 @@ class NullSerializer extends Serializer {
*/
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 toBinary(o: AnyRef): Array[Byte] = o match {
case null null

View file

@ -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")
}

View file

@ -121,7 +121,7 @@ final case class AdaptiveLoadBalancingRoutingLogic(system: ActorSystem, metricsS
@SerialVersionUID(1L)
final case class AdaptiveLoadBalancingPool(
metricsSelector: MetricsSelector = MixMetricsSelector,
override val nrOfInstances: Int = 0,
val nrOfInstances: Int = 0,
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
override val usePoolDispatcher: Boolean = false)
@ -203,7 +203,7 @@ final case class AdaptiveLoadBalancingPool(
@SerialVersionUID(1L)
final case class AdaptiveLoadBalancingGroup(
metricsSelector: MetricsSelector = MixMetricsSelector,
override val paths: immutable.Iterable[String] = Nil,
val paths: immutable.Iterable[String] = Nil,
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
extends Group {

View file

@ -26,14 +26,6 @@ import akka.cluster.InternalClusterAction.ExitingConfirmed
*/
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
// must be lazy because serializer is initialized from Cluster extension constructor
private lazy val GossipTimeToLive = Cluster(system).settings.GossipTimeToLive

View file

@ -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
* 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.
* Allows to define logic within the actor, which will respect the invocation-order-guarantee

View file

@ -217,10 +217,6 @@ abstract class UntypedPersistentActor extends UntypedActor with Eventsourced wit
def persistAll[A](events: JIterable[A], handler: Procedure[A]): Unit =
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
* persisted event.
@ -377,10 +373,6 @@ abstract class AbstractPersistentActor extends AbstractActor with PersistentActo
def persistAll[A](events: JIterable[A], handler: Procedure[A]): Unit =
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
* persisted event.
@ -413,10 +405,6 @@ abstract class AbstractPersistentActor extends AbstractActor with PersistentActo
def persistAllAsync[A](events: JIterable[A], handler: Procedure[A]): Unit =
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.
* Allows to define logic within the actor, which will respect the invocation-order-guarantee

View file

@ -41,11 +41,6 @@ class DeadlineFailureDetector(
acceptableHeartbeatPause = config.getMillisDuration("acceptable-heartbeat-pause"),
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(heartbeatInterval > Duration.Zero, "failure-detector.heartbeat-interval must be > 0 s")

View file

@ -84,7 +84,7 @@ final case class RemotingErrorEvent(cause: Throwable) extends RemotingLifecycleE
// For binary compatibility
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)
}
@ -99,13 +99,13 @@ final case class QuarantinedEvent(address: Address, longUid: Long) extends Remot
// 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)
@deprecated("Use long uid")
@deprecated("Use long uid", "2.4.x")
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)
}

View file

@ -18,16 +18,8 @@ import akka.serialization.SerializerWithStringManifest
class MessageContainerSerializer(val system: ExtendedActorSystem) extends BaseSerializer {
@deprecated("Use constructor with ExtendedActorSystem", "2.4")
def this() = this(null)
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 toBinary(obj: AnyRef): Array[Byte] = obj match {

View file

@ -40,17 +40,6 @@ object ProtobufSerializer {
*/
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 toByteArrayMethodBindingRef = new AtomicReference[Map[Class[_], Method]](Map.empty)

View file

@ -13,6 +13,7 @@ import akka.event.Logging
import akka.testkit.TestEvent
import akka.testkit.EventFilter
import org.testng.annotations.BeforeClass
import scala.concurrent.Await
trait ActorSystemLifecycle {
@ -31,8 +32,7 @@ trait ActorSystemLifecycle {
@AfterClass
def shutdownActorSystem(): Unit = {
try {
system.terminate()
system.awaitTermination(shutdownTimeout)
Await.ready(system.terminate(), shutdownTimeout)
} catch {
case _: TimeoutException
val msg = "Failed to stop [%s] within [%s] \n%s".format(system.name, shutdownTimeout,

View file

@ -413,7 +413,9 @@ class TcpSpec extends StreamSpec("akka.stream.materializer.subscription-timeout.
result.failed.futureValue shouldBe a[StreamTcpException]
binding.map(_.unbind()).recover { case NonFatal(_) () } foreach (_ system2.shutdown())
binding.map(_.unbind()).recover { case NonFatal(_) () }.foreach { _
shutdown(system2)
}
}
}

View file

@ -258,6 +258,80 @@ object MiMa extends AutoPlugin {
// object akka.stream.stage.StatefulStage#Finish does not have a correspondent in current version
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)
ProblemFilters.exclude[MissingClassProblem]("akka.persistence.Update"),
ProblemFilters.exclude[MissingClassProblem]("akka.persistence.Update$"),