= #17342 Make 2.4 binary compatible with 2.3

(cherry picked from commit 89af8bdb90)

* remove final identifier in serializers

i* revert/deprecate ProtobufSerializer.ARRAY_OF_BYTE_ARRAY

* adding back compatible empty constructor in serializers

* make FSM.State compatible

* add back ActorPath.ElementRegex

* revert SocketOption changes and add SocketOptionV2
  see a6d3704ef6

* problem filter for ActorSystem and ActorPath

* problem filter for ByteString

* problem filter for deprecated Timeout methods

* BalancingPool companion

* ask

* problem filter for ActorDSL

* event bus

* exclude hasSubscriptions

* exclude some problems in testkit

* boundAddress and addressFromSocketAddress

* Pool nrOfInstances

* PromiseActorRef

* check with 2.3.9

* migration guide note

* explicit exclude of final class problems
This commit is contained in:
Patrik Nordwall 2015-04-30 09:23:18 +02:00
parent 412491d277
commit b30e460be7
50 changed files with 1037 additions and 202 deletions

View file

@ -147,7 +147,7 @@ abstract class EventBusSpec(busName: String, conf: Config = ConfigFactory.empty(
object ActorEventBusSpec { object ActorEventBusSpec {
class MyActorEventBus(protected val system: ActorSystem) extends ActorEventBus class MyActorEventBus(protected val system: ActorSystem) extends ActorEventBus
with ActorClassification with ActorClassifier { with ManagedActorClassification with ActorClassifier {
type Event = Notification type Event = Notification
@ -179,7 +179,7 @@ class ActorEventBusSpec(conf: Config) extends EventBusSpec("ActorEventBus", conf
def disposeSubscriber(system: ActorSystem, subscriber: BusType#Subscriber): Unit = system.stop(subscriber) def disposeSubscriber(system: ActorSystem, subscriber: BusType#Subscriber): Unit = system.stop(subscriber)
// ActorClassification specific tests // ManagedActorClassification specific tests
"must unsubscribe subscriber when it terminates" in { "must unsubscribe subscriber when it terminates" in {
val a1 = createSubscriber(system.deadLetters) val a1 = createSubscriber(system.deadLetters)

View file

@ -11,6 +11,7 @@ import akka.actor.ActorRef
import akka.io.Udp._ import akka.io.Udp._
import akka.io.Inet._ import akka.io.Inet._
import akka.testkit.SocketUtil._ import akka.testkit.SocketUtil._
import java.net.DatagramSocket
class UdpIntegrationSpec extends AkkaSpec(""" class UdpIntegrationSpec extends AkkaSpec("""
akka.loglevel = INFO akka.loglevel = INFO
@ -86,7 +87,7 @@ class UdpIntegrationSpec extends AkkaSpec("""
"call SocketOption.afterConnect method after binding." in { "call SocketOption.afterConnect method after binding." in {
val commander = TestProbe() val commander = TestProbe()
val assertOption = AssertAfterConnect() val assertOption = AssertAfterChannelBind()
commander.send(IO(Udp), Bind(testActor, addresses(4), options = List(assertOption))) commander.send(IO(Udp), Bind(testActor, addresses(4), options = List(assertOption)))
commander.expectMsg(Bound(addresses(4))) commander.expectMsg(Bound(addresses(4)))
assert(assertOption.afterCalled === 1) assert(assertOption.afterCalled === 1)
@ -106,17 +107,17 @@ class UdpIntegrationSpec extends AkkaSpec("""
private case class AssertBeforeBind() extends SocketOption { private case class AssertBeforeBind() extends SocketOption {
var beforeCalled = 0 var beforeCalled = 0
override def beforeBind(c: DatagramChannel) = { override def beforeDatagramBind(ds: DatagramSocket): Unit = {
assert(!c.socket.isBound) assert(!ds.isBound)
beforeCalled += 1 beforeCalled += 1
} }
} }
private case class AssertAfterConnect() extends SocketOption { private case class AssertAfterChannelBind() extends SocketOptionV2 {
var afterCalled = 0 var afterCalled = 0
override def afterConnect(c: DatagramChannel) = { override def afterBind(s: DatagramSocket) = {
assert(c.socket.isBound) assert(s.isBound)
afterCalled += 1 afterCalled += 1
} }
} }

View file

@ -17,6 +17,9 @@ 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

View file

@ -122,18 +122,38 @@ object FSM {
/** Used by `forMax` to signal "cancel stateTimeout" */ /** Used by `forMax` to signal "cancel stateTimeout" */
private final val SomeMaxFiniteDuration = Some(Long.MaxValue.nanos) private final val SomeMaxFiniteDuration = Some(Long.MaxValue.nanos)
/**
* INTERNAL API
* Using a subclass for binary compatibility reasons
*/
private[akka] class SilentState[S, D](_stateName: S, _stateData: D, _timeout: Option[FiniteDuration], _stopReason: Option[Reason], _replies: List[Any])
extends State[S, D](_stateName, _stateData, _timeout, _stopReason, _replies) {
/**
* INTERNAL API
*/
private[akka] override def notifies: Boolean = false
override def copy(stateName: S = stateName, stateData: D = stateData, timeout: Option[FiniteDuration] = timeout, stopReason: Option[Reason] = stopReason, replies: List[Any] = replies): State[S, D] = {
new SilentState(stateName, stateData, timeout, stopReason, replies)
}
}
/** /**
* This captures all of the managed state of the [[akka.actor.FSM]]: the state * This captures all of the managed state of the [[akka.actor.FSM]]: the state
* name, the state data, possibly custom timeout, stop reason and replies * name, the state data, possibly custom timeout, stop reason and replies
* accumulated while processing the last message. * accumulated while processing the last message.
*/ */
final case class State[S, D](stateName: S, stateData: D, timeout: Option[FiniteDuration] = None, stopReason: Option[Reason] = None, replies: List[Any] = Nil)(private[akka] val notifies: Boolean = true) { case class State[S, D](stateName: S, stateData: D, timeout: Option[FiniteDuration] = None, stopReason: Option[Reason] = None, replies: List[Any] = Nil) {
/** /**
* Copy object and update values if needed. * INTERNAL API
*/ */
private[akka] def copy(stateName: S = stateName, stateData: D = stateData, timeout: Option[FiniteDuration] = timeout, stopReason: Option[Reason] = stopReason, replies: List[Any] = replies, notifies: Boolean = notifies): State[S, D] = { private[akka] def notifies: Boolean = true
State(stateName, stateData, timeout, stopReason, replies)(notifies)
// defined here to be able to override it in SilentState
def copy(stateName: S = stateName, stateData: D = stateData, timeout: Option[FiniteDuration] = timeout, stopReason: Option[Reason] = stopReason, replies: List[Any] = replies): State[S, D] = {
new State(stateName, stateData, timeout, stopReason, replies)
} }
/** /**
@ -174,8 +194,14 @@ object FSM {
copy(stopReason = Some(reason)) copy(stopReason = Some(reason))
} }
/**
* INTERNAL API.
*/
private[akka] def withNotification(notifies: Boolean): State[S, D] = { private[akka] def withNotification(notifies: Boolean): State[S, D] = {
copy(notifies = notifies) if (notifies)
State(stateName, stateData, timeout, stopReason, replies)
else
new SilentState(stateName, stateData, timeout, stopReason, replies)
} }
} }
@ -329,7 +355,7 @@ trait FSM[S, D] extends Actor with Listeners with ActorLogging {
* @param timeout state timeout for the initial state, overriding the default timeout for that state * @param timeout state timeout for the initial state, overriding the default timeout for that state
*/ */
final def startWith(stateName: S, stateData: D, timeout: Timeout = None): Unit = final def startWith(stateName: S, stateData: D, timeout: Timeout = None): Unit =
currentState = FSM.State(stateName, stateData, timeout)() currentState = FSM.State(stateName, stateData, timeout)
/** /**
* Produce transition to other state. * Produce transition to other state.
@ -341,7 +367,7 @@ trait FSM[S, D] extends Actor with Listeners with ActorLogging {
* @param nextStateName state designator for the next state * @param nextStateName state designator for the next state
* @return state transition descriptor * @return state transition descriptor
*/ */
final def goto(nextStateName: S): State = FSM.State(nextStateName, currentState.stateData)() final def goto(nextStateName: S): State = FSM.State(nextStateName, currentState.stateData)
/** /**
* Produce "empty" transition descriptor. * Produce "empty" transition descriptor.

View file

@ -180,6 +180,8 @@ trait SubchannelClassification { this: EventBus ⇒
* Expensive call! Avoid calling directly from event bus subscribe / unsubscribe. * Expensive call! Avoid calling directly from event bus subscribe / unsubscribe.
*/ */
private[akka] def hasSubscriptions(subscriber: Subscriber): Boolean = private[akka] def hasSubscriptions(subscriber: Subscriber): Boolean =
// FIXME binary incompatible, but I think it is safe to filter out this problem,
// since it is only called from new functionality in EventStreamUnsubscriber
cache.values exists { _ contains subscriber } cache.values exists { _ contains subscriber }
private def removeFromCache(changes: immutable.Seq[(Classifier, Set[Subscriber])]): Unit = private def removeFromCache(changes: immutable.Seq[(Classifier, Set[Subscriber])]): Unit =
@ -256,33 +258,33 @@ trait ScanningClassification { self: EventBus ⇒
* All subscribers will be watched by an [[akka.event.ActorClassificationUnsubscriber]] and unsubscribed when they terminate. * All subscribers will be watched by an [[akka.event.ActorClassificationUnsubscriber]] and unsubscribed when they terminate.
* The unsubscriber actor will not be stopped automatically, and if you want to stop using the bus you should stop it yourself. * The unsubscriber actor will not be stopped automatically, and if you want to stop using the bus you should stop it yourself.
*/ */
trait ActorClassification { this: ActorEventBus with ActorClassifier trait ManagedActorClassification { this: ActorEventBus with ActorClassifier
import scala.annotation.tailrec import scala.annotation.tailrec
protected def system: ActorSystem protected def system: ActorSystem
private class ActorClassificationMappings(val seqNr: Int, val backing: Map[ActorRef, immutable.TreeSet[ActorRef]]) { private class ManagedActorClassificationMappings(val seqNr: Int, val backing: Map[ActorRef, immutable.TreeSet[ActorRef]]) {
def get(monitored: ActorRef): immutable.TreeSet[ActorRef] = backing.getOrElse(monitored, empty) def get(monitored: ActorRef): immutable.TreeSet[ActorRef] = backing.getOrElse(monitored, empty)
def add(monitored: ActorRef, monitor: ActorRef) = { def add(monitored: ActorRef, monitor: ActorRef) = {
val watchers = backing.get(monitored).getOrElse(empty) + monitor val watchers = backing.get(monitored).getOrElse(empty) + monitor
new ActorClassificationMappings(seqNr + 1, backing.updated(monitored, watchers)) new ManagedActorClassificationMappings(seqNr + 1, backing.updated(monitored, watchers))
} }
def remove(monitored: ActorRef, monitor: ActorRef) = { def remove(monitored: ActorRef, monitor: ActorRef) = {
val monitors = backing.get(monitored).getOrElse(empty) - monitor val monitors = backing.get(monitored).getOrElse(empty) - monitor
new ActorClassificationMappings(seqNr + 1, backing.updated(monitored, monitors)) new ManagedActorClassificationMappings(seqNr + 1, backing.updated(monitored, monitors))
} }
def remove(monitored: ActorRef) = { def remove(monitored: ActorRef) = {
val v = backing - monitored val v = backing - monitored
new ActorClassificationMappings(seqNr + 1, v) new ManagedActorClassificationMappings(seqNr + 1, v)
} }
} }
private val mappings = new AtomicReference[ActorClassificationMappings]( private val mappings = new AtomicReference[ManagedActorClassificationMappings](
new ActorClassificationMappings(0, Map.empty[ActorRef, immutable.TreeSet[ActorRef]])) new ManagedActorClassificationMappings(0, Map.empty[ActorRef, immutable.TreeSet[ActorRef]]))
private val empty = immutable.TreeSet.empty[ActorRef] private val empty = immutable.TreeSet.empty[ActorRef]
@ -409,3 +411,113 @@ trait ActorClassification { this: ActorEventBus with ActorClassifier ⇒
true true
} }
} }
/**
* 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

@ -78,7 +78,7 @@ private[akka] object EventStreamUnsubscriber {
* *
* Watches all actors which subscribe on the given event stream, and unsubscribes them from it when they are Terminated. * Watches all actors which subscribe on the given event stream, and unsubscribes them from it when they are Terminated.
*/ */
private[akka] class ActorClassificationUnsubscriber(bus: ActorClassification, debug: Boolean) extends Actor with Stash { private[akka] class ActorClassificationUnsubscriber(bus: ManagedActorClassification, debug: Boolean) extends Actor with Stash {
import ActorClassificationUnsubscriber._ import ActorClassificationUnsubscriber._
@ -130,13 +130,13 @@ private[akka] object ActorClassificationUnsubscriber {
final case class Register(actor: ActorRef, seq: Int) final case class Register(actor: ActorRef, seq: Int)
final case class Unregister(actor: ActorRef, seq: Int) final case class Unregister(actor: ActorRef, seq: Int)
def start(system: ActorSystem, bus: ActorClassification, debug: Boolean = false) = { def start(system: ActorSystem, bus: ManagedActorClassification, debug: Boolean = false) = {
val debug = system.settings.config.getBoolean("akka.actor.debug.event-stream") val debug = system.settings.config.getBoolean("akka.actor.debug.event-stream")
system.asInstanceOf[ExtendedActorSystem] system.asInstanceOf[ExtendedActorSystem]
.systemActorOf(props(bus, debug), "actorClassificationUnsubscriber-" + unsubscribersCount.incrementAndGet()) .systemActorOf(props(bus, debug), "actorClassificationUnsubscriber-" + unsubscribersCount.incrementAndGet())
} }
private def props(eventBus: ActorClassification, debug: Boolean) = Props(classOf[ActorClassificationUnsubscriber], eventBus, debug) private def props(eventBus: ManagedActorClassification, debug: Boolean) = Props(classOf[ActorClassificationUnsubscriber], eventBus, debug)
} }

View file

@ -11,6 +11,11 @@ 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.
@ -20,7 +25,12 @@ import scala.annotation.tailrec
* The debug flag in the constructor toggles if operations on this EventStream should also be published * The debug flag in the constructor toggles if operations on this EventStream should also be published
* as Debug-Events * as Debug-Events
*/ */
class EventStream(sys: ActorSystem, private val debug: Boolean = false) extends LoggingBus with SubchannelClassification { class EventStream(sys: ActorSystem, private val debug: Boolean) extends LoggingBus with SubchannelClassification {
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[_]
@ -36,7 +46,8 @@ class EventStream(sys: ActorSystem, private val debug: Boolean = false) extends
protected def classify(event: AnyRef): Class[_] = event.getClass protected def classify(event: AnyRef): Class[_] = event.getClass
protected def publish(event: AnyRef, subscriber: ActorRef) = { protected def publish(event: AnyRef, subscriber: ActorRef) = {
subscriber ! event if (sys == null && subscriber.isTerminated) unsubscribe(subscriber)
else subscriber ! event
} }
override def subscribe(subscriber: ActorRef, channel: Class[_]): Boolean = { override def subscribe(subscriber: ActorRef, channel: Class[_]): Boolean = {
@ -65,14 +76,18 @@ class EventStream(sys: ActorSystem, private val debug: Boolean = false) extends
* ''Must'' be called after actor system is "ready". * ''Must'' be called after actor system is "ready".
* Starts system actor that takes care of unsubscribing subscribers that have terminated. * Starts system actor that takes care of unsubscribing subscribers that have terminated.
*/ */
def startUnsubscriber() = EventStreamUnsubscriber.start(sys, this) def startUnsubscriber(): Unit =
// sys may be null for backwards compatibility reasons
if (sys ne null) EventStreamUnsubscriber.start(sys, this)
/** /**
* INTERNAL API * INTERNAL API
*/ */
@tailrec @tailrec
final private[akka] def initUnsubscriber(unsubscriber: ActorRef): Boolean = { final private[akka] def initUnsubscriber(unsubscriber: ActorRef): Boolean = {
initiallySubscribedOrUnsubscriber.get match { // sys may be null for backwards compatibility reasons
if (sys eq null) false
else initiallySubscribedOrUnsubscriber.get match {
case value @ Left(subscribers) case value @ Left(subscribers)
if (initiallySubscribedOrUnsubscriber.compareAndSet(value, Right(unsubscriber))) { if (initiallySubscribedOrUnsubscriber.compareAndSet(value, Right(unsubscriber))) {
if (debug) publish(Logging.Debug(simpleName(this), this.getClass, "initialized unsubscriber to: " + unsubscriber + ", registering " + subscribers.size + " initial subscribers with it")) if (debug) publish(Logging.Debug(simpleName(this), this.getClass, "initialized unsubscriber to: " + unsubscriber + ", registering " + subscribers.size + " initial subscribers with it"))
@ -95,7 +110,8 @@ class EventStream(sys: ActorSystem, private val debug: Boolean = false) extends
*/ */
@tailrec @tailrec
private def registerWithUnsubscriber(subscriber: ActorRef): Unit = { private def registerWithUnsubscriber(subscriber: ActorRef): Unit = {
initiallySubscribedOrUnsubscriber.get match { // sys may be null for backwards compatibility reasons
if (sys ne null) initiallySubscribedOrUnsubscriber.get match {
case value @ Left(subscribers) case value @ Left(subscribers)
if (!initiallySubscribedOrUnsubscriber.compareAndSet(value, Left(subscribers + subscriber))) if (!initiallySubscribedOrUnsubscriber.compareAndSet(value, Left(subscribers + subscriber)))
registerWithUnsubscriber(subscriber) registerWithUnsubscriber(subscriber)
@ -114,7 +130,8 @@ class EventStream(sys: ActorSystem, private val debug: Boolean = false) extends
*/ */
@tailrec @tailrec
private def unregisterIfNoMoreSubscribedChannels(subscriber: ActorRef): Unit = { private def unregisterIfNoMoreSubscribedChannels(subscriber: ActorRef): Unit = {
initiallySubscribedOrUnsubscriber.get match { // sys may be null for backwards compatibility reasons
if (sys ne null) initiallySubscribedOrUnsubscriber.get match {
case value @ Left(subscribers) case value @ Left(subscribers)
if (!initiallySubscribedOrUnsubscriber.compareAndSet(value, Left(subscribers - subscriber))) if (!initiallySubscribedOrUnsubscriber.compareAndSet(value, Left(subscribers - subscriber)))
unregisterIfNoMoreSubscribedChannels(subscriber) unregisterIfNoMoreSubscribedChannels(subscriber)

View file

@ -183,18 +183,51 @@ abstract class ScanningEventBus[E, S, C] extends EventBus[E, S, C] {
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.ManagedActorClassification]]
* An EventBus where the Subscribers are ActorRefs and the Classifier is ActorRef
* Means that ActorRefs "listen" to other ActorRefs
* E is the Event type
*/
abstract class ManagedActorEventBus[E](system: ActorSystem) extends EventBus[E, ActorRef, ActorRef] {
private val bus = new akka.event.ActorEventBus with akka.event.ManagedActorClassification with akka.event.ActorClassifier {
type Event = E
override val system = ManagedActorEventBus.this.system
override protected def mapSize: Int = ManagedActorEventBus.this.mapSize
override protected def classify(event: E): ActorRef =
ManagedActorEventBus.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)
}
/** /**
* Java API: See documentation for [[akka.event.ActorClassification]] * Java API: See documentation for [[akka.event.ActorClassification]]
* An EventBus where the Subscribers are ActorRefs and the Classifier is ActorRef * An EventBus where the Subscribers are ActorRefs and the Classifier is ActorRef
* Means that ActorRefs "listen" to other ActorRefs * Means that ActorRefs "listen" to other ActorRefs
* E is the Event type * E is the Event type
*/ */
abstract class ActorEventBus[E](system: ActorSystem) extends EventBus[E, ActorRef, ActorRef] { @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 { private val bus = new akka.event.ActorEventBus with akka.event.ActorClassification with akka.event.ActorClassifier {
type Event = E type Event = E
override val system = ActorEventBus.this.system
override protected def mapSize: Int = ActorEventBus.this.mapSize override protected def mapSize: Int = ActorEventBus.this.mapSize
override protected def classify(event: E): ActorRef = override protected def classify(event: E): ActorRef =

View file

@ -4,6 +4,9 @@
package akka.io package akka.io
import java.nio.channels.{ DatagramChannel, SocketChannel, ServerSocketChannel } import java.nio.channels.{ DatagramChannel, SocketChannel, ServerSocketChannel }
import java.net.DatagramSocket
import java.net.ServerSocket
import java.net.Socket
object Inet { object Inet {
@ -16,37 +19,57 @@ object Inet {
/** /**
* Action to be taken for this option before bind() is called * Action to be taken for this option before bind() is called
*/ */
def beforeBind(ds: DatagramChannel): Unit = () def beforeDatagramBind(ds: DatagramSocket): Unit = ()
/** /**
* Action to be taken for this option before bind() is called * Action to be taken for this option before bind() is called
*/ */
def beforeBind(ss: ServerSocketChannel): Unit = () def beforeServerSocketBind(ss: ServerSocket): Unit = ()
/** /**
* Action to be taken for this option before bind() is called * Action to be taken for this option before calling connect()
*/ */
def beforeBind(s: SocketChannel): Unit = () def beforeConnect(s: Socket): Unit = ()
/** /**
* Action to be taken for this option after connect returned (i.e. on * Action to be taken for this option after connect returned (i.e. on
* the slave socket for servers). * the slave socket for servers).
*/ */
def afterConnect(c: DatagramChannel): Unit = () def afterConnect(s: Socket): Unit = ()
/**
* Action to be taken for this option after connect returned (i.e. on
* the slave socket for servers).
*/
def afterConnect(c: ServerSocketChannel): Unit = ()
/**
* Action to be taken for this option after connect returned (i.e. on
* the slave socket for servers).
*/
def afterConnect(c: SocketChannel): Unit = ()
} }
/**
* Java API: AbstractSocketOption is a package of data (from the user) and associated
* behavior (how to apply that to a channel).
*/
abstract class AbstractSocketOption extends SocketOption
trait SocketOptionV2 extends SocketOption {
/**
* Action to be taken for this option after connect returned (i.e. on
* the slave socket for servers).
*/
def afterBind(s: DatagramSocket): Unit = ()
/**
* Action to be taken for this option after connect returned (i.e. on
* the slave socket for servers).
*/
def afterBind(s: ServerSocket): Unit = ()
/**
* Action to be taken for this option after connect returned (i.e. on
* the slave socket for servers).
*/
def afterConnect(s: DatagramSocket): Unit = ()
}
/**
* Java API
*/
abstract class AbstractSocketOptionV2 extends SocketOptionV2
/** /**
* DatagramChannel creation behavior. * DatagramChannel creation behavior.
*/ */
@ -76,9 +99,9 @@ object Inet {
*/ */
final case class ReceiveBufferSize(size: Int) extends SocketOption { final case class ReceiveBufferSize(size: Int) extends SocketOption {
require(size > 0, "ReceiveBufferSize must be > 0") require(size > 0, "ReceiveBufferSize must be > 0")
override def beforeBind(c: ServerSocketChannel): Unit = c.socket.setReceiveBufferSize(size) override def beforeServerSocketBind(s: ServerSocket): Unit = s.setReceiveBufferSize(size)
override def beforeBind(c: DatagramChannel): Unit = c.socket.setReceiveBufferSize(size) override def beforeDatagramBind(s: DatagramSocket): Unit = s.setReceiveBufferSize(size)
override def beforeBind(c: SocketChannel): Unit = c.socket.setReceiveBufferSize(size) override def beforeConnect(s: Socket): Unit = s.setReceiveBufferSize(size)
} }
// server socket options // server socket options
@ -89,9 +112,9 @@ object Inet {
* For more information see [[java.net.Socket.setReuseAddress]] * For more information see [[java.net.Socket.setReuseAddress]]
*/ */
final case class ReuseAddress(on: Boolean) extends SocketOption { final case class ReuseAddress(on: Boolean) extends SocketOption {
override def beforeBind(c: ServerSocketChannel): Unit = c.socket.setReuseAddress(on) override def beforeServerSocketBind(s: ServerSocket): Unit = s.setReuseAddress(on)
override def beforeBind(c: DatagramChannel): Unit = c.socket.setReuseAddress(on) override def beforeDatagramBind(s: DatagramSocket): Unit = s.setReuseAddress(on)
override def beforeBind(c: SocketChannel): Unit = c.socket.setReuseAddress(on) override def beforeConnect(s: Socket): Unit = s.setReuseAddress(on)
} }
/** /**
@ -101,8 +124,7 @@ object Inet {
*/ */
final case class SendBufferSize(size: Int) extends SocketOption { final case class SendBufferSize(size: Int) extends SocketOption {
require(size > 0, "SendBufferSize must be > 0") require(size > 0, "SendBufferSize must be > 0")
override def afterConnect(c: DatagramChannel): Unit = c.socket.setSendBufferSize(size) override def afterConnect(s: Socket): Unit = s.setSendBufferSize(size)
override def afterConnect(c: SocketChannel): Unit = c.socket.setSendBufferSize(size)
} }
/** /**
@ -114,8 +136,7 @@ object Inet {
*/ */
final case class TrafficClass(tc: Int) extends SocketOption { final case class TrafficClass(tc: Int) extends SocketOption {
require(0 <= tc && tc <= 255, "TrafficClass needs to be in the interval [0, 255]") require(0 <= tc && tc <= 255, "TrafficClass needs to be in the interval [0, 255]")
override def afterConnect(c: DatagramChannel): Unit = c.socket.setTrafficClass(tc) override def afterConnect(s: Socket): Unit = s.setTrafficClass(tc)
override def afterConnect(c: SocketChannel): Unit = c.socket.setTrafficClass(tc)
} }
} }
@ -185,43 +206,4 @@ object Inet {
def trafficClass(tc: Int) = TrafficClass(tc) def trafficClass(tc: Int) = TrafficClass(tc)
} }
/**
* Java API: AbstractSocketOption is a package of data (from the user) and associated
* behavior (how to apply that to a channel).
*/
abstract class AbstractSocketOption extends SocketOption {
/**
* Action to be taken for this option before bind() is called
*/
override def beforeBind(ds: DatagramChannel): Unit = ()
/**
* Action to be taken for this option before bind() is called
*/
override def beforeBind(ss: ServerSocketChannel): Unit = ()
/**
* Action to be taken for this option before bind() is called
*/
override def beforeBind(s: SocketChannel): Unit = ()
/**
* Action to be taken for this option after connect returned (i.e. on
* the slave socket for servers).
*/
override def afterConnect(c: DatagramChannel): Unit = ()
/**
* Action to be taken for this option after connect returned (i.e. on
* the slave socket for servers).
*/
override def afterConnect(c: ServerSocketChannel): Unit = ()
/**
* Action to be taken for this option after connect returned (i.e. on
* the slave socket for servers).
*/
override def afterConnect(c: SocketChannel): Unit = ()
}
} }

View file

@ -7,7 +7,6 @@ package akka.io
import java.net.InetSocketAddress import java.net.InetSocketAddress
import java.net.Socket import java.net.Socket
import akka.ConfigurationException import akka.ConfigurationException
import java.nio.channels.SocketChannel
import akka.io.Inet._ import akka.io.Inet._
import com.typesafe.config.Config import com.typesafe.config.Config
import scala.concurrent.duration._ import scala.concurrent.duration._
@ -58,7 +57,7 @@ object Tcp extends ExtensionId[TcpExt] with ExtensionIdProvider {
* For more information see [[java.net.Socket.setKeepAlive]] * For more information see [[java.net.Socket.setKeepAlive]]
*/ */
final case class KeepAlive(on: Boolean) extends SocketOption { final case class KeepAlive(on: Boolean) extends SocketOption {
override def afterConnect(c: SocketChannel): Unit = c.socket.setKeepAlive(on) override def afterConnect(s: Socket): Unit = s.setKeepAlive(on)
} }
/** /**
@ -69,7 +68,7 @@ object Tcp extends ExtensionId[TcpExt] with ExtensionIdProvider {
* For more information see [[java.net.Socket.setOOBInline]] * For more information see [[java.net.Socket.setOOBInline]]
*/ */
final case class OOBInline(on: Boolean) extends SocketOption { final case class OOBInline(on: Boolean) extends SocketOption {
override def afterConnect(c: SocketChannel): Unit = c.socket.setOOBInline(on) override def afterConnect(s: Socket): Unit = s.setOOBInline(on)
} }
// SO_LINGER is handled by the Close code // SO_LINGER is handled by the Close code
@ -83,7 +82,7 @@ object Tcp extends ExtensionId[TcpExt] with ExtensionIdProvider {
* For more information see [[java.net.Socket.setTcpNoDelay]] * For more information see [[java.net.Socket.setTcpNoDelay]]
*/ */
final case class TcpNoDelay(on: Boolean) extends SocketOption { final case class TcpNoDelay(on: Boolean) extends SocketOption {
override def afterConnect(c: SocketChannel): Unit = c.socket.setTcpNoDelay(on) override def afterConnect(s: Socket): Unit = s.setTcpNoDelay(on)
} }
} }

View file

@ -187,7 +187,7 @@ private[io] abstract class TcpConnection(val tcp: TcpExt, val channel: SocketCha
// continue anyway // continue anyway
log.debug("Could not enable TcpNoDelay: {}", e.getMessage) log.debug("Could not enable TcpNoDelay: {}", e.getMessage)
} }
options.foreach(_.afterConnect(channel)) options.foreach(_.afterConnect(channel.socket))
commander ! Connected( commander ! Connected(
channel.socket.getRemoteSocketAddress.asInstanceOf[InetSocketAddress], channel.socket.getRemoteSocketAddress.asInstanceOf[InetSocketAddress],

View file

@ -49,7 +49,7 @@ private[io] class TcpListener(selectorRouter: ActorRef,
val localAddress = val localAddress =
try { try {
val socket = channel.socket val socket = channel.socket
bind.options.foreach(_.beforeBind(channel)) bind.options.foreach(_.beforeServerSocketBind(socket))
socket.bind(bind.localAddress, bind.backlog) socket.bind(bind.localAddress, bind.backlog)
val ret = socket.getLocalSocketAddress match { val ret = socket.getLocalSocketAddress match {
case isa: InetSocketAddress isa case isa: InetSocketAddress isa
@ -57,7 +57,10 @@ private[io] class TcpListener(selectorRouter: ActorRef,
} }
channelRegistry.register(channel, if (bind.pullMode) 0 else SelectionKey.OP_ACCEPT) channelRegistry.register(channel, if (bind.pullMode) 0 else SelectionKey.OP_ACCEPT)
log.debug("Successfully bound to {}", ret) log.debug("Successfully bound to {}", ret)
bind.options.foreach(_.afterConnect(channel)) bind.options.foreach {
case o: Inet.SocketOptionV2 o.afterBind(channel.socket)
case _
}
ret ret
} catch { } catch {
case NonFatal(e) case NonFatal(e)

View file

@ -32,7 +32,7 @@ private[io] class TcpOutgoingConnection(_tcp: TcpExt,
context.watch(commander) // sign death pact context.watch(commander) // sign death pact
options.foreach(_.beforeBind(channel)) options.foreach(_.beforeConnect(channel.socket))
localAddress.foreach(channel.socket.bind) localAddress.foreach(channel.socket.bind)
channelRegistry.register(channel, 0) channelRegistry.register(channel, 0)
timeout foreach context.setReceiveTimeout //Initiate connection timeout if supplied timeout foreach context.setReceiveTimeout //Initiate connection timeout if supplied

View file

@ -3,8 +3,8 @@
*/ */
package akka.io package akka.io
import java.net.DatagramSocket
import java.net.InetSocketAddress import java.net.InetSocketAddress
import java.nio.channels.DatagramChannel
import com.typesafe.config.Config import com.typesafe.config.Config
import scala.collection.immutable import scala.collection.immutable
import akka.io.Inet.{ SoJavaFactories, SocketOption } import akka.io.Inet.{ SoJavaFactories, SocketOption }
@ -180,7 +180,7 @@ object Udp extends ExtensionId[UdpExt] with ExtensionIdProvider {
* For more information see [[java.net.DatagramSocket#setBroadcast]] * For more information see [[java.net.DatagramSocket#setBroadcast]]
*/ */
final case class Broadcast(on: Boolean) extends SocketOption { final case class Broadcast(on: Boolean) extends SocketOption {
override def beforeBind(c: DatagramChannel): Unit = c.socket.setBroadcast(on) override def beforeDatagramBind(s: DatagramSocket): Unit = s.setBroadcast(on)
} }
} }

View file

@ -57,7 +57,7 @@ private[io] class UdpConnection(udpConn: UdpConnectedExt,
channel = DatagramChannel.open channel = DatagramChannel.open
channel.configureBlocking(false) channel.configureBlocking(false)
val socket = channel.socket val socket = channel.socket
options.foreach(_.beforeBind(channel)) options.foreach(_.beforeDatagramBind(socket))
localAddress foreach socket.bind localAddress foreach socket.bind
channel.connect(remoteAddress) channel.connect(remoteAddress)
channelRegistry.register(channel, OP_READ) channelRegistry.register(channel, OP_READ)
@ -67,7 +67,10 @@ private[io] class UdpConnection(udpConn: UdpConnectedExt,
def receive = { def receive = {
case registration: ChannelRegistration case registration: ChannelRegistration
options.foreach(_.afterConnect(channel)) options.foreach {
case v2: Inet.SocketOptionV2 v2.afterConnect(channel.socket)
case _
}
commander ! Connected commander ! Connected
context.become(connected(registration), discardOld = true) context.become(connected(registration), discardOld = true)
} }

View file

@ -40,7 +40,7 @@ private[io] class UdpListener(val udp: UdpExt,
val localAddress = val localAddress =
try { try {
val socket = channel.socket val socket = channel.socket
bind.options.foreach(_.beforeBind(channel)) bind.options.foreach(_.beforeDatagramBind(socket))
socket.bind(bind.localAddress) socket.bind(bind.localAddress)
val ret = socket.getLocalSocketAddress match { val ret = socket.getLocalSocketAddress match {
case isa: InetSocketAddress isa case isa: InetSocketAddress isa
@ -48,7 +48,10 @@ private[io] class UdpListener(val udp: UdpExt,
} }
channelRegistry.register(channel, OP_READ) channelRegistry.register(channel, OP_READ)
log.debug("Successfully bound to [{}]", ret) log.debug("Successfully bound to [{}]", ret)
bind.options.foreach(_.afterConnect(channel)) bind.options.foreach {
case o: Inet.SocketOptionV2 o.afterBind(channel.socket)
case _
}
ret ret
} catch { } catch {
case NonFatal(e) case NonFatal(e)

View file

@ -23,7 +23,8 @@ private[io] class UdpSender(val udp: UdpExt,
val channel = { val channel = {
val datagramChannel = DatagramChannel.open val datagramChannel = DatagramChannel.open
datagramChannel.configureBlocking(false) datagramChannel.configureBlocking(false)
options foreach { _.beforeBind(datagramChannel) } val socket = datagramChannel.socket
options foreach { _.beforeDatagramBind(socket) }
datagramChannel datagramChannel
} }
@ -31,7 +32,10 @@ private[io] class UdpSender(val udp: UdpExt,
def receive: Receive = { def receive: Receive = {
case registration: ChannelRegistration case registration: ChannelRegistration
options.foreach(_.afterConnect(channel)) options.foreach {
case v2: Inet.SocketOptionV2 v2.afterConnect(channel.socket)
case _
}
commander ! SimpleSenderReady commander ! SimpleSenderReady
context.become(sendHandlers(registration)) context.become(sendHandlers(registration))
} }

View file

@ -72,9 +72,10 @@ trait AskSupport {
* }}} * }}}
* *
*/ */
def ask(actorRef: ActorRef, message: Any)(implicit timeout: Timeout): Future[Any] = actorRef ? message def ask(actorRef: ActorRef, message: Any)(implicit timeout: Timeout): Future[Any] =
actorRef.internalAsk(message, timeout, ActorRef.noSender)
def ask(actorRef: ActorRef, message: Any, sender: ActorRef)(implicit timeout: Timeout): Future[Any] = def ask(actorRef: ActorRef, message: Any, sender: ActorRef)(implicit timeout: Timeout): Future[Any] =
actorRef.?(message)(timeout, sender) actorRef.internalAsk(message, timeout, sender)
/** /**
* Import this implicit conversion to gain `?` and `ask` methods on * Import this implicit conversion to gain `?` and `ask` methods on
@ -120,9 +121,24 @@ trait AskSupport {
* }}} * }}}
* *
*/ */
def ask(actorSelection: ActorSelection, message: Any)(implicit timeout: Timeout): Future[Any] = actorSelection ? message def ask(actorSelection: ActorSelection, message: Any)(implicit timeout: Timeout): Future[Any] =
actorSelection.internalAsk(message, timeout, ActorRef.noSender)
def ask(actorSelection: ActorSelection, message: Any, sender: ActorRef)(implicit timeout: Timeout): Future[Any] = def ask(actorSelection: ActorSelection, message: Any, sender: ActorRef)(implicit timeout: Timeout): Future[Any] =
actorSelection.?(message)(timeout, sender) actorSelection.internalAsk(message, timeout, sender)
}
object AskableActorRef {
/**
* INTERNAL API: for binary compatibility
*/
private[pattern] def ask$extension(actorRef: ActorRef, message: Any, timeout: Timeout): Future[Any] =
actorRef.internalAsk(message, timeout, ActorRef.noSender)
/**
* INTERNAL API: for binary compatibility
*/
private[pattern] def $qmark$extension(actorRef: ActorRef, message: Any, timeout: Timeout): Future[Any] =
actorRef.internalAsk(message, timeout, ActorRef.noSender)
} }
/* /*
@ -130,7 +146,28 @@ trait AskSupport {
*/ */
final class AskableActorRef(val actorRef: ActorRef) extends AnyVal { final class AskableActorRef(val actorRef: ActorRef) extends AnyVal {
def ask(message: Any)(implicit timeout: Timeout, sender: ActorRef = Actor.noSender): Future[Any] = actorRef match { /**
* INTERNAL API: for binary compatibility
*/
protected def ask(message: Any, timeout: Timeout): Future[Any] =
internalAsk(message, timeout, ActorRef.noSender)
def ask(message: Any)(implicit timeout: Timeout, sender: ActorRef = Actor.noSender): Future[Any] =
internalAsk(message, timeout, sender)
/**
* INTERNAL API: for binary compatibility
*/
protected def ?(message: Any)(implicit timeout: Timeout): Future[Any] =
internalAsk(message, timeout, ActorRef.noSender)
def ?(message: Any)(implicit timeout: Timeout, sender: ActorRef = Actor.noSender): Future[Any] =
internalAsk(message, timeout, sender)
/**
* INTERNAL API: for binary compatibility
*/
private[pattern] def internalAsk(message: Any, timeout: Timeout, sender: ActorRef) = actorRef match {
case ref: InternalActorRef if ref.isTerminated case ref: InternalActorRef if ref.isTerminated
actorRef ! message actorRef ! message
Future.failed[Any](new AskTimeoutException(s"""Recipient[$actorRef] had already been terminated. Sender[$sender] sent the message of type "${message.getClass.getName}".""")) Future.failed[Any](new AskTimeoutException(s"""Recipient[$actorRef] had already been terminated. Sender[$sender] sent the message of type "${message.getClass.getName}"."""))
@ -145,7 +182,20 @@ final class AskableActorRef(val actorRef: ActorRef) extends AnyVal {
case _ Future.failed[Any](new IllegalArgumentException(s"""Unsupported recipient ActorRef type, question not sent to [$actorRef]. Sender[$sender] sent the message of type "${message.getClass.getName}".""")) case _ Future.failed[Any](new IllegalArgumentException(s"""Unsupported recipient ActorRef type, question not sent to [$actorRef]. Sender[$sender] sent the message of type "${message.getClass.getName}"."""))
} }
def ?(message: Any)(implicit timeout: Timeout, sender: ActorRef = Actor.noSender): Future[Any] = ask(message)(timeout, sender) }
object AskableActorSelection {
/**
* INTERNAL API: for binary compatibility
*/
private[pattern] def ask$extension(actorSel: ActorSelection, message: Any, timeout: Timeout): Future[Any] =
actorSel.internalAsk(message, timeout, ActorRef.noSender)
/**
* INTERNAL API: for binary compatibility
*/
private[pattern] def $qmark$extension(actorSel: ActorSelection, message: Any, timeout: Timeout): Future[Any] =
actorSel.internalAsk(message, timeout, ActorRef.noSender)
} }
/* /*
@ -153,7 +203,28 @@ final class AskableActorRef(val actorRef: ActorRef) extends AnyVal {
*/ */
final class AskableActorSelection(val actorSel: ActorSelection) extends AnyVal { final class AskableActorSelection(val actorSel: ActorSelection) extends AnyVal {
def ask(message: Any)(implicit timeout: Timeout, sender: ActorRef = Actor.noSender): Future[Any] = actorSel.anchor match { /**
* INTERNAL API: for binary compatibility
*/
protected def ask(message: Any, timeout: Timeout): Future[Any] =
internalAsk(message, timeout, ActorRef.noSender)
def ask(message: Any)(implicit timeout: Timeout, sender: ActorRef = Actor.noSender): Future[Any] =
internalAsk(message, timeout, sender)
/**
* INTERNAL API: for binary compatibility
*/
protected def ?(message: Any)(implicit timeout: Timeout): Future[Any] =
internalAsk(message, timeout, ActorRef.noSender)
def ?(message: Any)(implicit timeout: Timeout, sender: ActorRef = Actor.noSender): Future[Any] =
internalAsk(message, timeout, sender)
/**
* INTERNAL API: for binary compatibility
*/
private[pattern] def internalAsk(message: Any, timeout: Timeout, sender: ActorRef): Future[Any] = actorSel.anchor match {
case ref: InternalActorRef case ref: InternalActorRef
if (timeout.duration.length <= 0) if (timeout.duration.length <= 0)
Future.failed[Any]( Future.failed[Any](
@ -165,8 +236,6 @@ final class AskableActorSelection(val actorSel: ActorSelection) extends AnyVal {
} }
case _ Future.failed[Any](new IllegalArgumentException(s"""Unsupported recipient ActorRef type, question not sent to [$actorSel]. Sender[$sender] sent the message of type "${message.getClass.getName}".""")) case _ Future.failed[Any](new IllegalArgumentException(s"""Unsupported recipient ActorRef type, question not sent to [$actorSel]. Sender[$sender] sent the message of type "${message.getClass.getName}"."""))
} }
def ?(message: Any)(implicit timeout: Timeout, sender: ActorRef = Actor.noSender): Future[Any] = ask(message)(timeout, sender)
} }
/** /**
@ -180,6 +249,9 @@ 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
@ -344,4 +416,8 @@ private[akka] object PromiseActorRef {
result.future onComplete { _ try a.stop() finally f.cancel() } result.future onComplete { _ try a.stop() finally f.cancel() }
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)
} }

View file

@ -66,7 +66,7 @@ private[akka] final class BalancingRoutingLogic extends RoutingLogic {
*/ */
@SerialVersionUID(1L) @SerialVersionUID(1L)
final case class BalancingPool( final case class BalancingPool(
nrOfInstances: Int, override 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 {
@ -101,7 +101,7 @@ final case class BalancingPool(
override private[akka] def newRoutee(routeeProps: Props, context: ActorContext): Routee = { override private[akka] def newRoutee(routeeProps: Props, context: ActorContext): Routee = {
val rawDeployPath = context.self.path.elements.drop(1).mkString("/", "/", "") val rawDeployPath = context.self.path.elements.drop(1).mkString("/", "/", "")
val deployPath = BalancingPool.invalidConfigKeyChars.foldLeft(rawDeployPath) { (replaced, c) val deployPath = BalancingPoolDeploy.invalidConfigKeyChars.foldLeft(rawDeployPath) { (replaced, c)
replaced.replace(c, '_') replaced.replace(c, '_')
} }
val dispatcherId = s"BalancingPool-$deployPath" val dispatcherId = s"BalancingPool-$deployPath"
@ -113,7 +113,7 @@ final case class BalancingPool(
val deployDispatcherConfigPath = s"akka.actor.deployment.$deployPath.pool-dispatcher" val deployDispatcherConfigPath = s"akka.actor.deployment.$deployPath.pool-dispatcher"
val systemConfig = context.system.settings.config val systemConfig = context.system.settings.config
val dispatcherConfig = context.system.dispatchers.config(dispatcherId, val dispatcherConfig = context.system.dispatchers.config(dispatcherId,
// use the user defined 'pool-dispatcher' config as fallback, if any // use the user defined 'pool-dispatcher' config as fallback, if any
if (systemConfig.hasPath(deployDispatcherConfigPath)) systemConfig.getConfig(deployDispatcherConfigPath) if (systemConfig.hasPath(deployDispatcherConfigPath)) systemConfig.getConfig(deployDispatcherConfigPath)
else ConfigFactory.empty) else ConfigFactory.empty)
@ -151,6 +151,10 @@ final case class BalancingPool(
} }
object BalancingPool { /**
private val invalidConfigKeyChars = List('$', '@', ':') * INTERNAL API
* Can't be in the `BalancingPool` companion for binary compatibility reasons.
*/
private[akka] object BalancingPoolDeploy {
val invalidConfigKeyChars = List('$', '@', ':')
} }

View file

@ -58,7 +58,7 @@ final class BroadcastRoutingLogic extends RoutingLogic {
*/ */
@SerialVersionUID(1L) @SerialVersionUID(1L)
final case class BroadcastPool( final case class BroadcastPool(
nrOfInstances: Int, override val resizer: Option[Resizer] = None, override 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)

View file

@ -260,7 +260,7 @@ final case class ConsistentHashingRoutingLogic(
*/ */
@SerialVersionUID(1L) @SerialVersionUID(1L)
final case class ConsistentHashingPool( final case class ConsistentHashingPool(
val nrOfInstances: Int, override 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,

View file

@ -59,7 +59,7 @@ final class RandomRoutingLogic extends RoutingLogic {
*/ */
@SerialVersionUID(1L) @SerialVersionUID(1L)
final case class RandomPool( final case class RandomPool(
val nrOfInstances: Int, override val resizer: Option[Resizer] = None, override 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)

View file

@ -64,7 +64,7 @@ final class RoundRobinRoutingLogic extends RoutingLogic {
*/ */
@SerialVersionUID(1L) @SerialVersionUID(1L)
final case class RoundRobinPool( final case class RoundRobinPool(
val nrOfInstances: Int, override val resizer: Option[Resizer] = None, override 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)

View file

@ -102,7 +102,10 @@ private[akka] class RoutedActorCell(
_router = routerConfig.createRouter(system) _router = routerConfig.createRouter(system)
routerConfig match { routerConfig match {
case pool: Pool case pool: Pool
val nrOfRoutees = pool.nrOfInstances(system) // 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
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

View file

@ -177,6 +177,9 @@ 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
*/ */

View file

@ -93,7 +93,7 @@ private[akka] final case class ScatterGatherFirstCompletedRoutees(
*/ */
@SerialVersionUID(1L) @SerialVersionUID(1L)
final case class ScatterGatherFirstCompletedPool( final case class ScatterGatherFirstCompletedPool(
val nrOfInstances: Int, override val resizer: Option[Resizer] = None, override 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,

View file

@ -174,7 +174,7 @@ class SmallestMailboxRoutingLogic extends RoutingLogic {
*/ */
@SerialVersionUID(1L) @SerialVersionUID(1L)
final case class SmallestMailboxPool( final case class SmallestMailboxPool(
val nrOfInstances: Int, override val resizer: Option[Resizer] = None, override 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)

View file

@ -120,7 +120,7 @@ private[akka] final case class TailChoppingRoutees(
*/ */
@SerialVersionUID(1L) @SerialVersionUID(1L)
final case class TailChoppingPool( final case class TailChoppingPool(
val nrOfInstances: Int, override val resizer: Option[Resizer] = None, override 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,

View file

@ -73,7 +73,7 @@ trait BaseSerializer extends Serializer {
/** /**
* Actor system which is required by most serializer implementations. * Actor system which is required by most serializer implementations.
*/ */
val system: ExtendedActorSystem def system: ExtendedActorSystem
/** /**
* Configuration namespace of serialization identifiers in the `reference.conf`. * Configuration namespace of serialization identifiers in the `reference.conf`.
* *
@ -88,7 +88,12 @@ trait BaseSerializer extends Serializer {
* *
* See [[Serializer#identifier()]]. * See [[Serializer#identifier()]].
*/ */
final override val identifier: Int = override val identifier: Int = identifierFromConfig
/**
* INTERNAL API
*/
private[akka] def identifierFromConfig: Int =
system.settings.config.getInt(s"""${SerializationIdentifiers}."${getClass.getName}"""") system.settings.config.getInt(s"""${SerializationIdentifiers}."${getClass.getName}"""")
} }
@ -146,6 +151,14 @@ 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] = {
@ -180,6 +193,15 @@ class NullSerializer extends Serializer {
* (just returns the byte array unchanged/uncopied) * (just returns the byte array unchanged/uncopied)
*/ */
class ByteArraySerializer(val system: ExtendedActorSystem) extends BaseSerializer { class ByteArraySerializer(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) 4
else identifierFromConfig
def includeManifest: Boolean = false def includeManifest: Boolean = false
def toBinary(o: AnyRef) = o match { def toBinary(o: AnyRef) = o match {
case null null case null null

View file

@ -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,
val nrOfInstances: Int = 0, override 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)

View file

@ -24,6 +24,14 @@ import scala.concurrent.duration.Deadline
*/ */
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

View file

@ -132,7 +132,7 @@ final case class AdaptiveLoadBalancingRoutingLogic(system: ActorSystem, metricsS
@deprecated("Superseded by akka.cluster.metrics (in akka-cluster-metrics jar)", "2.4") @deprecated("Superseded by akka.cluster.metrics (in akka-cluster-metrics jar)", "2.4")
final case class AdaptiveLoadBalancingPool( final case class AdaptiveLoadBalancingPool(
metricsSelector: MetricsSelector = MixMetricsSelector, metricsSelector: MetricsSelector = MixMetricsSelector,
val nrOfInstances: Int = 0, override 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)

View file

@ -39,7 +39,7 @@ import akka.event.japi.ScanningEventBus;
//#scanning-bus //#scanning-bus
//#actor-bus //#actor-bus
import akka.event.japi.ActorEventBus; import akka.event.japi.ManagedActorEventBus;
//#actor-bus //#actor-bus
@ -232,7 +232,7 @@ public class EventBusDocTest {
static static
//#actor-bus //#actor-bus
public class ActorBusImpl extends ActorEventBus<Notification> { public class ActorBusImpl extends ManagedActorEventBus<Notification> {
// the ActorSystem will be used for book-keeping operations, such as subscribers terminating // the ActorSystem will be used for book-keeping operations, such as subscribers terminating
public ActorBusImpl(ActorSystem system) { public ActorBusImpl(ActorSystem system) {
@ -304,7 +304,7 @@ public class EventBusDocTest {
} }
@Test @Test
public void demonstrateActorClassification() { public void demonstrateManagedActorClassification() {
//#actor-bus-test //#actor-bus-test
ActorRef observer1 = new JavaTestKit(system).getRef(); ActorRef observer1 = new JavaTestKit(system).getRef();
ActorRef observer2 = new JavaTestKit(system).getRef(); ActorRef observer2 = new JavaTestKit(system).getRef();

View file

@ -115,7 +115,7 @@ type :class:`ActorRef`.
This classification requires an :class:`ActorSystem` in order to perform book-keeping This classification requires an :class:`ActorSystem` in order to perform book-keeping
operations related to the subscribers being Actors, which can terminate without first operations related to the subscribers being Actors, which can terminate without first
unsubscribing from the EventBus. ActorClassification maitains a system Actor which unsubscribing from the EventBus. ManagedActorClassification maitains a system Actor which
takes care of unsubscribing terminated actors automatically. takes care of unsubscribing terminated actors automatically.
The necessary methods to be implemented are illustrated with the following example: The necessary methods to be implemented are illustrated with the following example:

View file

@ -11,6 +11,19 @@ When migrating from earlier versions you should first follow the instructions fo
migrating :ref:`1.3.x to 2.0.x <migration-2.0>` and then :ref:`2.0.x to 2.1.x <migration-2.1>` migrating :ref:`1.3.x to 2.0.x <migration-2.0>` and then :ref:`2.0.x to 2.1.x <migration-2.1>`
and then :ref:`2.1.x to 2.2.x <migration-2.2>` and then :ref:`2.2.x to 2.3.x <migration-2.3>`. and then :ref:`2.1.x to 2.2.x <migration-2.2>` and then :ref:`2.2.x to 2.3.x <migration-2.3>`.
Binary Compatibility
====================
Akka 2.4.x is backwards binary compatible with previous 2.3.x versions apart from the following
exceptions. This means that the new JARs are a drop-in replacement for the old one
(but not the other way around) as long as your build does not enable the inliner (Scala-only restriction).
The following parts are not binary compatible with 2.3.x:
* akka-testkit and akka-remote-testkit
* experimental modules, such as akka-persistence and akka-contrib
* features, classes, methods that were deprecated in 2.3.x and removed in 2.4.x
Advanced Notice: TypedActors will go away Advanced Notice: TypedActors will go away
======================================== ========================================
@ -31,13 +44,14 @@ In earlier versions of Akka `TestKit.remaining` returned the default timeout con
AssertionError if called outside of within. The old behavior however can still be achieved by AssertionError if called outside of within. The old behavior however can still be achieved by
calling `TestKit.remainingOrDefault` instead. calling `TestKit.remainingOrDefault` instead.
EventStream and ActorClassification EventBus now require an ActorSystem EventStream and ManagedActorClassification EventBus now require an ActorSystem
======================================================================= ==============================================================================
Both the ``EventStream`` (:ref:`Scala <event-stream-scala>`, :ref:`Java <event-stream-java>`) and the Both the ``EventStream`` (:ref:`Scala <event-stream-scala>`, :ref:`Java <event-stream-java>`) and the
``ActorClassification`` Event Bus (:ref:`Scala <actor-classification-scala>`, :ref:`Java <actor-classification-java>`) now ``ManagedActorClassification``, ``ManagedActorEventBus`` (:ref:`Scala <actor-classification-scala>`, :ref:`Java <actor-classification-java>`) now
require an ``ActorSystem`` to properly operate. The reason for that is moving away from stateful internal lifecycle checks require an ``ActorSystem`` to properly operate. The reason for that is moving away from stateful internal lifecycle checks
to a fully reactive model for unsubscribing actors that have ``Terminated``. to a fully reactive model for unsubscribing actors that have ``Terminated``. Therefore the ``ActorClassification``
and ``ActorEventBus`` was deprecated and replaced by ``ManagedActorClassification`` and ``ManagedActorEventBus``
If you have implemented a custom event bus, you will need to pass in the actor system through the constructor now: If you have implemented a custom event bus, you will need to pass in the actor system through the constructor now:
@ -63,25 +77,6 @@ Which turns out to be useful in many systems where same-state transitions actual
In case you do *not* want to trigger a state transition event when effectively performing an ``X->X`` transition, use ``stay()`` instead. In case you do *not* want to trigger a state transition event when effectively performing an ``X->X`` transition, use ``stay()`` instead.
More control over Channel properties in Akka-IO
===============================================
Method signatures for ``SocketOption`` have been changed to take a channel instead of a socket. The channel's socket
can be retrieved by calling ``channel.socket``. This allows for accessing new NIO features in Java 7.
======================================== =====================================
2.3 2.4
======================================== =====================================
``beforeDatagramBind(DatagramSocket)`` ``beforeBind(DatagramChannel)``
``beforeServerSocketBind(ServerSocket)`` ``beforeBind(ServerSocketChannel)``
``beforeConnect(Socket)`` ``beforeBind(SocketChannel)``
\ ``afterConnect(DatagramChannel)``
\ ``afterConnect(ServerSocketChannel)``
``afterConnect(Socket)`` ``afterConnect(SocketChannel)``
======================================== =====================================
A new class ``DatagramChannelCreator`` which extends ``SocketOption`` has been added. ``DatagramChannelCreator`` can be used for
custom ``DatagramChannel`` creation logic. This allows for opening IPv6 multicast datagram channels.
Cluster Sharding Entry Path Change Cluster Sharding Entry Path Change
================================== ==================================
Previously in ``2.3.x`` entries were direct children of the local ``ShardRegion``. In examples the ``persistenceId`` of entries Previously in ``2.3.x`` entries were direct children of the local ``ShardRegion``. In examples the ``persistenceId`` of entries

View file

@ -121,12 +121,12 @@ object EventBusDocSpec {
//#actor-bus //#actor-bus
import akka.event.ActorEventBus import akka.event.ActorEventBus
import akka.event.ActorClassification import akka.event.ManagedActorClassification
import akka.event.ActorClassifier import akka.event.ActorClassifier
final case class Notification(ref: ActorRef, id: Int) final case class Notification(ref: ActorRef, id: Int)
class ActorBusImpl(val system: ActorSystem) extends ActorEventBus with ActorClassifier with ActorClassification { class ActorBusImpl(val system: ActorSystem) extends ActorEventBus with ActorClassifier with ManagedActorClassification {
type Event = Notification type Event = Notification
// is used for extracting the classifier from the incoming events // is used for extracting the classifier from the incoming events
@ -179,7 +179,7 @@ class EventBusDocSpec extends AkkaSpec {
//#scanning-bus-test //#scanning-bus-test
} }
"demonstrate ActorClassification" in { "demonstrate ManagedActorClassification" in {
//#actor-bus-test //#actor-bus-test
val observer1 = TestProbe().ref val observer1 = TestProbe().ref
val observer2 = TestProbe().ref val observer2 = TestProbe().ref

View file

@ -115,7 +115,7 @@ type :class:`ActorRef`.
This classification requires an :class:`ActorSystem` in order to perform book-keeping This classification requires an :class:`ActorSystem` in order to perform book-keeping
operations related to the subscribers being Actors, which can terminate without first operations related to the subscribers being Actors, which can terminate without first
unsubscribing from the EventBus. ActorClassification maitains a system Actor which unsubscribing from the EventBus. ManagedActorClassification maitains a system Actor which
takes care of unsubscribing terminated actors automatically. takes care of unsubscribing terminated actors automatically.
The necessary methods to be implemented are illustrated with the following example: The necessary methods to be implemented are illustrated with the following example:

View file

@ -231,7 +231,10 @@ private[remote] class Remoting(_system: ExtendedActorSystem, _provider: RemoteAc
private[akka] def boundAddresses: Map[String, Set[Address]] = { private[akka] def boundAddresses: Map[String, Set[Address]] = {
transportMapping.map { transportMapping.map {
case (scheme, transports) case (scheme, transports)
scheme -> transports.map { case (transport, _) transport.boundAddress } scheme -> transports.flatMap {
// Need to do like this for binary compatibility reasons
case (t, _) Option(t.boundAddress)
}
} }
} }
} }

View file

@ -29,6 +29,14 @@ private[akka] class DaemonMsgCreateSerializer(val system: ExtendedActorSystem) e
import ProtobufSerializer.deserializeActorRef import ProtobufSerializer.deserializeActorRef
import Deploy.NoDispatcherGiven import Deploy.NoDispatcherGiven
@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) 3
else identifierFromConfig
def includeManifest: Boolean = false def includeManifest: Boolean = false
lazy val serialization = SerializationExtension(system) lazy val serialization = SerializationExtension(system)

View file

@ -17,6 +17,14 @@ import akka.serialization.BaseSerializer
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)
// 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 {

View file

@ -39,6 +39,17 @@ 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)
override def includeManifest: Boolean = true override def includeManifest: Boolean = true
@ -52,8 +63,9 @@ class ProtobufSerializer(val system: ExtendedActorSystem) extends BaseSerializer
parsingMethodBinding.get(clazz) match { parsingMethodBinding.get(clazz) match {
case Some(cachedParsingMethod) cachedParsingMethod case Some(cachedParsingMethod) cachedParsingMethod
case None case None
import ProtobufSerializer.ARRAY_OF_BYTE_ARRAY val unCachedParsingMethod =
val unCachedParsingMethod = if (method eq null) clazz.getDeclaredMethod("parseFrom", ARRAY_OF_BYTE_ARRAY: _*) else method if (method eq null) clazz.getDeclaredMethod("parseFrom", ProtobufSerializer.ARRAY_OF_BYTE_ARRAY: _*)
else method
if (parsingMethodBindingRef.compareAndSet(parsingMethodBinding, parsingMethodBinding.updated(clazz, unCachedParsingMethod))) if (parsingMethodBindingRef.compareAndSet(parsingMethodBinding, parsingMethodBinding.updated(clazz, unCachedParsingMethod)))
unCachedParsingMethod unCachedParsingMethod
else else

View file

@ -90,7 +90,18 @@ abstract class AbstractTransportAdapter(protected val wrappedTransport: Transpor
} yield (augmentScheme(listenAddress), upstreamListenerPromise) } yield (augmentScheme(listenAddress), upstreamListenerPromise)
} }
override def boundAddress: Address = wrappedTransport.boundAddress /**
* INTERNAL API
* @return
* The address this Transport is listening to.
*/
private[akka] def boundAddress: Address = wrappedTransport match {
// Need to do like this in the backport of #15007 to 2.3.x for binary compatibility reasons
case t: AbstractTransportAdapter t.boundAddress
case t: netty.NettyTransport t.boundAddress
case t: TestTransport t.boundAddress
case _ null
}
override def associate(remoteAddress: Address): Future[AssociationHandle] = { override def associate(remoteAddress: Address): Future[AssociationHandle] = {
// Prepare a future, and pass its promise to the manager // Prepare a future, and pass its promise to the manager

View file

@ -111,7 +111,8 @@ class TestTransport(
(_) registry.logActivity(ShutdownAttempt(localAddress))) (_) registry.logActivity(ShutdownAttempt(localAddress)))
override def listen: Future[(Address, Promise[AssociationEventListener])] = listenBehavior(()) override def listen: Future[(Address, Promise[AssociationEventListener])] = listenBehavior(())
override def boundAddress = localAddress // Need to do like this for binary compatibility reasons
private[akka] def boundAddress = localAddress
override def associate(remoteAddress: Address): Future[AssociationHandle] = associateBehavior(remoteAddress) override def associate(remoteAddress: Address): Future[AssociationHandle] = associateBehavior(remoteAddress)
override def shutdown(): Future[Boolean] = shutdownBehavior(()) override def shutdown(): Future[Boolean] = shutdownBehavior(())

View file

@ -104,11 +104,8 @@ trait Transport {
*/ */
def listen: Future[(Address, Promise[AssociationEventListener])] def listen: Future[(Address, Promise[AssociationEventListener])]
/** // Need to do like this for binary compatibility reasons
* @return // def boundAddress: Address
* The address this Transport is listening to.
*/
def boundAddress: Address
/** /**
* Asynchronously opens a logical duplex link between two Transport Entities over a network. It could be backed by a * Asynchronously opens a logical duplex link between two Transport Entities over a network. It could be backed by a

View file

@ -243,6 +243,11 @@ private[transport] object NettyTransport {
hostName.getOrElse(sa.getAddress.getHostAddress), port.getOrElse(sa.getPort))) // perhaps use getHostString in jdk 1.7 hostName.getOrElse(sa.getAddress.getHostAddress), port.getOrElse(sa.getPort))) // perhaps use getHostString in jdk 1.7
case _ None case _ None
} }
// Need to do like this for binary compatibility reasons
def addressFromSocketAddress(addr: SocketAddress, schemeIdentifier: String, systemName: String,
hostName: Option[String]): Option[Address] =
addressFromSocketAddress(addr, schemeIdentifier, systemName, hostName, port = None)
} }
// FIXME: Split into separate UDP and TCP classes // FIXME: Split into separate UDP and TCP classes
@ -434,7 +439,8 @@ class NettyTransport(val settings: NettyTransportSettings, val system: ExtendedA
} }
} }
override def boundAddress = boundTo // Need to do like this for binary compatibility reasons
private[akka] def boundAddress = boundTo
override def associate(remoteAddress: Address): Future[AssociationHandle] = { override def associate(remoteAddress: Address): Future[AssociationHandle] = {
if (!serverChannel.isBound) Future.failed(new NettyTransportException("Transport is not bound")) if (!serverChannel.isBound) Future.failed(new NettyTransportException("Transport is not bound"))

View file

@ -18,6 +18,7 @@ import java.net.InetAddress;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.net.NetworkInterface; import java.net.NetworkInterface;
import java.net.StandardProtocolFamily; import java.net.StandardProtocolFamily;
import java.net.DatagramSocket;
import java.nio.channels.DatagramChannel; import java.nio.channels.DatagramChannel;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
@ -34,7 +35,7 @@ public class JavaUdpMulticast {
//#inet6-protocol-family //#inet6-protocol-family
//#multicast-group //#multicast-group
public static class MulticastGroup extends Inet.AbstractSocketOption { public static class MulticastGroup extends Inet.AbstractSocketOptionV2 {
private String address; private String address;
private String interf; private String interf;
@ -44,11 +45,11 @@ public class JavaUdpMulticast {
} }
@Override @Override
public void afterConnect(DatagramChannel c) { public void afterBind(DatagramSocket s) {
try { try {
InetAddress group = InetAddress.getByName(address); InetAddress group = InetAddress.getByName(address);
NetworkInterface networkInterface = NetworkInterface.getByName(interf); NetworkInterface networkInterface = NetworkInterface.getByName(interf);
c.join(group, networkInterface); s.getChannel().join(group, networkInterface);
} catch (Exception ex) { } catch (Exception ex) {
System.out.println("Unable to join multicast group."); System.out.println("Unable to join multicast group.");
} }

View file

@ -5,10 +5,11 @@
package docs.io package docs.io
import java.net.{InetAddress, InetSocketAddress, NetworkInterface, StandardProtocolFamily} import java.net.{InetAddress, InetSocketAddress, NetworkInterface, StandardProtocolFamily}
import java.net.DatagramSocket
import java.nio.channels.DatagramChannel import java.nio.channels.DatagramChannel
import akka.actor.{Actor, ActorLogging, ActorRef} import akka.actor.{Actor, ActorLogging, ActorRef}
import akka.io.Inet.{DatagramChannelCreator, SocketOption} import akka.io.Inet.{DatagramChannelCreator, SocketOption, SocketOptionV2}
import akka.io.{IO, Udp} import akka.io.{IO, Udp}
import akka.util.ByteString import akka.util.ByteString
@ -20,11 +21,11 @@ final case class Inet6ProtocolFamily() extends DatagramChannelCreator {
//#inet6-protocol-family //#inet6-protocol-family
//#multicast-group //#multicast-group
final case class MulticastGroup(address: String, interface: String) extends SocketOption { final case class MulticastGroup(address: String, interface: String) extends SocketOptionV2 {
override def afterConnect(c: DatagramChannel) { override def afterBind(s: DatagramSocket) {
val group = InetAddress.getByName(address) val group = InetAddress.getByName(address)
val networkInterface = NetworkInterface.getByName(interface) val networkInterface = NetworkInterface.getByName(interface)
c.join(group, networkInterface) s.getChannel.join(group, networkInterface)
} }
} }
//#multicast-group //#multicast-group

View file

@ -62,7 +62,7 @@ class TestFSMRef[S, D, T <: Actor](
* and stop handling. * and stop handling.
*/ */
def setState(stateName: S = fsm.stateName, stateData: D = fsm.stateData, timeout: FiniteDuration = null, stopReason: Option[FSM.Reason] = None) { def setState(stateName: S = fsm.stateName, stateData: D = fsm.stateData, timeout: FiniteDuration = null, stopReason: Option[FSM.Reason] = None) {
fsm.applyState(FSM.State(stateName, stateData, Option(timeout), stopReason)()) fsm.applyState(FSM.State(stateName, stateData, Option(timeout), stopReason))
} }
/** /**

View file

@ -27,7 +27,7 @@ object AkkaBuild extends Build {
override def buildLoaders = BuildLoader.transform(Sample.buildTransformer) :: Nil override def buildLoaders = BuildLoader.transform(Sample.buildTransformer) :: Nil
val enableMiMa = false val enableMiMa = true
val parallelExecutionByDefault = false // TODO: enable this once we're sure it doesn not break things val parallelExecutionByDefault = false // TODO: enable this once we're sure it doesn not break things
@ -347,13 +347,7 @@ object AkkaBuild extends Build {
def akkaPreviousArtifact(id: String): Def.Initialize[Option[sbt.ModuleID]] = Def.setting { def akkaPreviousArtifact(id: String): Def.Initialize[Option[sbt.ModuleID]] = Def.setting {
if (enableMiMa) { if (enableMiMa) {
// Note: This is a little gross because we don't have a 2.3.0 release on Scala 2.11.x val version: String = "2.3.10" // FIXME
// This should be expanded if there are more deviations.
val version: String =
scalaBinaryVersion.value match {
case "2.11" => "2.3.2"
case _ => "2.3.0"
}
val fullId = crossVersion.value match { val fullId = crossVersion.value match {
case _ : CrossVersion.Binary => id + "_" + scalaBinaryVersion.value case _ : CrossVersion.Binary => id + "_" + scalaBinaryVersion.value
case _ : CrossVersion.Full => id + "_" + scalaVersion.value case _ : CrossVersion.Full => id + "_" + scalaVersion.value

View file

@ -17,10 +17,506 @@ object MiMa extends AutoPlugin {
previousArtifact := None, previousArtifact := None,
binaryIssueFilters ++= mimaIgnoredProblems binaryIssueFilters ++= mimaIgnoredProblems
) )
case class FilterAnyProblem(name: String) extends com.typesafe.tools.mima.core.ProblemFilter {
import com.typesafe.tools.mima.core._
override def apply(p: Problem): Boolean = p match {
case t: TemplateProblem => t.ref.fullName != name && t.ref.fullName != (name + '$')
case m: MemberProblem => m.ref.owner.fullName != name && m.ref.owner.fullName != (name + '$')
}
}
val mimaIgnoredProblems = { val mimaIgnoredProblems = {
import com.typesafe.tools.mima.core._
Seq( Seq(
// add filters here, see release-2.3 branch FilterAnyProblem("akka.remote.testconductor.Terminate"),
FilterAnyProblem("akka.remote.testconductor.TerminateMsg"),
ProblemFilters.exclude[MissingMethodProblem]("akka.remote.testconductor.Conductor.shutdown"),
ProblemFilters.exclude[MissingMethodProblem]("akka.remote.testkit.MultiNodeSpec.akka$remote$testkit$MultiNodeSpec$$deployer"),
FilterAnyProblem("akka.remote.EndpointManager$Pass"),
FilterAnyProblem("akka.remote.EndpointManager$EndpointRegistry"),
FilterAnyProblem("akka.remote.EndpointWriter"),
FilterAnyProblem("akka.remote.EndpointWriter$StopReading"),
FilterAnyProblem("akka.remote.EndpointWriter$State"),
FilterAnyProblem("akka.remote.EndpointWriter$TakeOver"),
// Change of internal message by #15109
ProblemFilters.exclude[MissingMethodProblem]("akka.remote.ReliableDeliverySupervisor#GotUid.copy"),
ProblemFilters.exclude[MissingMethodProblem]("akka.remote.ReliableDeliverySupervisor#GotUid.this"),
ProblemFilters.exclude[MissingTypesProblem]("akka.remote.ReliableDeliverySupervisor$GotUid$"),
ProblemFilters.exclude[MissingMethodProblem]("akka.remote.ReliableDeliverySupervisor#GotUid.apply"),
// Change of private method to protected by #15212
ProblemFilters.exclude[MissingMethodProblem]("akka.persistence.snapshot.local.LocalSnapshotStore.akka$persistence$snapshot$local$LocalSnapshotStore$$save"),
// Changes in akka-stream-experimental are not binary compatible - still source compatible (2.3.3 -> 2.3.4)
// Adding `PersistentActor.persistAsync`
// Adding `PersistentActor.defer`
// Changes in akka-persistence-experimental in #13944
// Changes in private LevelDB Store by #13962
// Renamed `processorId` to `persistenceId`
ProblemFilters.excludePackage("akka.persistence"),
// Adding wildcardFanOut to internal message ActorSelectionMessage by #13992
FilterAnyProblem("akka.actor.ActorSelectionMessage$"),
FilterAnyProblem("akka.actor.ActorSelectionMessage"),
ProblemFilters.exclude[MissingMethodProblem]("akka.remote.ContainerFormats#SelectionEnvelopeOrBuilder.hasWildcardFanOut"),
ProblemFilters.exclude[MissingMethodProblem]("akka.remote.ContainerFormats#SelectionEnvelopeOrBuilder.getWildcardFanOut"),
// Adding expectMsg overload to testkit #15425
ProblemFilters.exclude[MissingMethodProblem]("akka.testkit.TestKitBase.expectMsg"),
// Adding akka.japi.Option.getOrElse #15383
ProblemFilters.exclude[MissingMethodProblem]("akka.japi.Option.getOrElse"),
// Change to internal API to fix #15991
ProblemFilters.exclude[MissingClassProblem]("akka.io.TcpConnection$UpdatePendingWrite$"),
ProblemFilters.exclude[MissingClassProblem]("akka.io.TcpConnection$UpdatePendingWrite"),
// Change to optimize use of ForkJoin with Akka's Mailbox
ProblemFilters.exclude[MissingMethodProblem]("akka.dispatch.Mailbox.status"),
// Changes introduced to internal remoting actors by #16623
ProblemFilters.exclude[MissingMethodProblem]("akka.remote.ReliableDeliverySupervisor.unstashAcks"),
ProblemFilters.exclude[MissingMethodProblem]("akka.remote.ReliableDeliverySupervisor.pendingAcks_="),
ProblemFilters.exclude[MissingMethodProblem]("akka.remote.ReliableDeliverySupervisor.pendingAcks"),
ProblemFilters.exclude[MissingMethodProblem]("akka.remote.ReliableDeliverySupervisor.scheduleAutoResend"),
ProblemFilters.exclude[MissingMethodProblem]("akka.remote.ReliableDeliverySupervisor.autoResendTimer_="),
ProblemFilters.exclude[MissingMethodProblem]("akka.remote.ReliableDeliverySupervisor.rescheduleAutoResend"),
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.remote.ReliableDeliverySupervisor.autoResendTimer"),
ProblemFilters.exclude[MissingMethodProblem]("akka.remote.ReliableDeliverySupervisor.lastCumulativeAck"),
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.remote.ReliableDeliverySupervisor.bailoutAt"),
ProblemFilters.exclude[MissingMethodProblem]("akka.remote.ReliableDeliverySupervisor.lastCumulativeAck_="),
// Change to improve cluster heartbeat sender, #16638
FilterAnyProblem("akka.cluster.HeartbeatNodeRing"),
FilterAnyProblem("akka.cluster.ClusterHeartbeatSenderState"),
//Changes to improve BatchingExecutor, bugfix #16327
ProblemFilters.exclude[MissingMethodProblem]("akka.dispatch.BatchingExecutor.resubmitOnBlock"),
ProblemFilters.exclude[FinalClassProblem]("akka.dispatch.BatchingExecutor$Batch"),
ProblemFilters.exclude[MissingMethodProblem]("akka.dispatch.BatchingExecutor#Batch.initial"),
ProblemFilters.exclude[MissingMethodProblem]("akka.dispatch.BatchingExecutor#Batch.blockOn"),
ProblemFilters.exclude[FinalMethodProblem]("akka.dispatch.BatchingExecutor#Batch.run"),
ProblemFilters.exclude[MissingMethodProblem]("akka.dispatch.BatchingExecutor#Batch.akka$dispatch$BatchingExecutor$Batch$$parentBlockContext_="),
ProblemFilters.exclude[MissingMethodProblem]("akka.dispatch.BatchingExecutor#Batch.this"),
// Exclude observations from downed, #13875
ProblemFilters.exclude[MissingMethodProblem]("akka.cluster.ClusterEvent.diffReachable"),
ProblemFilters.exclude[MissingMethodProblem]("akka.cluster.ClusterEvent.diffSeen"),
ProblemFilters.exclude[MissingMethodProblem]("akka.cluster.ClusterEvent.diffUnreachable"),
ProblemFilters.exclude[MissingMethodProblem]("akka.cluster.ClusterEvent.diffRolesLeader"),
ProblemFilters.exclude[MissingMethodProblem]("akka.cluster.ClusterEvent.diffLeader"),
ProblemFilters.exclude[MissingMethodProblem]("akka.cluster.Gossip.convergence"),
ProblemFilters.exclude[MissingMethodProblem]("akka.cluster.Gossip.akka$cluster$Gossip$$convergenceMemberStatus"),
ProblemFilters.exclude[MissingMethodProblem]("akka.cluster.Gossip.isLeader"),
ProblemFilters.exclude[MissingMethodProblem]("akka.cluster.Gossip.leader"),
ProblemFilters.exclude[MissingMethodProblem]("akka.cluster.Gossip.roleLeader"),
// copied everything above from release-2.3 branch
// final case classes
ProblemFilters.exclude[FinalClassProblem]("akka.dispatch.ThreadPoolConfig"),
ProblemFilters.exclude[FinalClassProblem]("akka.dispatch.UnboundedDequeBasedMailbox"),
ProblemFilters.exclude[FinalClassProblem]("akka.dispatch.CachingConfig$ValuePathEntry"),
ProblemFilters.exclude[FinalClassProblem]("akka.dispatch.MonitorableThreadFactory"),
ProblemFilters.exclude[FinalClassProblem]("akka.dispatch.ThreadPoolConfigBuilder"),
ProblemFilters.exclude[FinalClassProblem]("akka.dispatch.DefaultDispatcherPrerequisites"),
ProblemFilters.exclude[FinalClassProblem]("akka.dispatch.BoundedMailbox"),
ProblemFilters.exclude[FinalClassProblem]("akka.dispatch.UnboundedMailbox"),
ProblemFilters.exclude[FinalClassProblem]("akka.dispatch.SingleConsumerOnlyUnboundedMailbox"),
ProblemFilters.exclude[FinalClassProblem]("akka.dispatch.CachingConfig$StringPathEntry"),
ProblemFilters.exclude[FinalClassProblem]("akka.dispatch.sysmsg.Supervise"),
ProblemFilters.exclude[FinalClassProblem]("akka.dispatch.sysmsg.Recreate"),
ProblemFilters.exclude[FinalClassProblem]("akka.dispatch.sysmsg.Resume"),
ProblemFilters.exclude[FinalClassProblem]("akka.dispatch.sysmsg.Failed"),
ProblemFilters.exclude[FinalClassProblem]("akka.dispatch.sysmsg.DeathWatchNotification"),
ProblemFilters.exclude[FinalClassProblem]("akka.dispatch.sysmsg.Create"),
ProblemFilters.exclude[FinalClassProblem]("akka.dispatch.sysmsg.Suspend"),
ProblemFilters.exclude[FinalClassProblem]("akka.dispatch.sysmsg.Unwatch"),
ProblemFilters.exclude[FinalClassProblem]("akka.dispatch.sysmsg.Terminate"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.AddressTerminated"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.FSM$Event"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.SuppressedDeadLetter"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.FSM$LogEntry"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.FSM$CurrentState"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.StopChild"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.ContextualTypedActorFactory"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.Status$Failure"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.FSM$Transition"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.FSM$SubscribeTransitionCallBack"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.SelectChildPattern"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.SerializedActorRef"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.TypedActor$SerializedMethodCall"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.Status$Success"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.FSM$UnsubscribeTransitionCallBack"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.PostRestartException"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.FSM$StopEvent"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.ActorKilledException"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.ChildRestartStats"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.ActorNotFound"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.TypedProps"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.SchedulerException"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.DeathPactException"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.FSM$Timer"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.Identify"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.InvalidMessageException"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.Terminated"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.PreRestartException"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.ActorIdentity"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.TypedActor$MethodCall"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.TypedActor$SerializedTypedActorInvocationHandler"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.IllegalActorStateException"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.InvalidActorNameException"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.SelectChildName"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.FSM$Failure"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.UnhandledMessage"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.DeadLetter"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.FSM$TimeoutMarker"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.dungeon.ChildrenContainer$Recreation"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.dungeon.ChildrenContainer$TerminatingChildrenContainer"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.dungeon.ChildrenContainer$Creation"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.dsl.Inbox$Select"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.dsl.Inbox$StartWatch"),
ProblemFilters.exclude[FinalClassProblem]("akka.actor.dsl.Inbox$Get"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.Tcp$Received"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.Udp$Send"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.TcpConnection$UpdatePendingWriteAndThen"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.Tcp$Write"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.Tcp$CommandFailed"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.Udp$Bound"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.TcpConnection$ConnectionInfo"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.Tcp$ErrorClosed"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.UdpConnected$Send"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.UdpConnected$Received"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.Udp$CommandFailed"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.SelectionHandler$Retry"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.Tcp$WriteFile"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.Tcp$Bound"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.UdpConnected$CommandFailed"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.Tcp$Register"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.Tcp$ResumeAccepting"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.SelectionHandler$WorkerForCommand"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.TcpConnection$CloseInformation"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.TcpConnection$WriteFileFailed"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.TcpListener$RegisterIncoming"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.Tcp$Connect"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.Tcp$Bind"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.Udp$Received"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.Tcp$Connected"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.UdpConnected$Connect"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.Udp$Bind"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.Tcp$CompoundWrite"),
ProblemFilters.exclude[FinalClassProblem]("akka.io.TcpListener$FailedRegisterIncoming"),
ProblemFilters.exclude[FinalClassProblem]("akka.event.Logging$InitializeLogger"),
ProblemFilters.exclude[FinalClassProblem]("akka.pattern.PromiseActorRef$StoppedWithPath"),
ProblemFilters.exclude[FinalClassProblem]("akka.serialization.Serialization$Information"),
ProblemFilters.exclude[FinalClassProblem]("akka.util.WildcardTree"),
ProblemFilters.exclude[FinalClassProblem]("akka.routing.AddRoutee"),
ProblemFilters.exclude[FinalClassProblem]("akka.routing.ConsistentRoutee"),
ProblemFilters.exclude[FinalClassProblem]("akka.routing.SeveralRoutees"),
ProblemFilters.exclude[FinalClassProblem]("akka.routing.ScatterGatherFirstCompletedRoutees"),
ProblemFilters.exclude[FinalClassProblem]("akka.routing.Deafen"),
ProblemFilters.exclude[FinalClassProblem]("akka.routing.Listen"),
ProblemFilters.exclude[FinalClassProblem]("akka.routing.AdjustPoolSize"),
ProblemFilters.exclude[FinalClassProblem]("akka.routing.ActorSelectionRoutee"),
ProblemFilters.exclude[FinalClassProblem]("akka.routing.Broadcast"),
ProblemFilters.exclude[FinalClassProblem]("akka.routing.RemoveRoutee"),
ProblemFilters.exclude[FinalClassProblem]("akka.routing.ActorRefRoutee"),
ProblemFilters.exclude[FinalClassProblem]("akka.routing.Routees"),
ProblemFilters.exclude[FinalClassProblem]("akka.routing.WithListeners"),
ProblemFilters.exclude[FinalClassProblem]("akka.testkit.TestEvent$Mute"),
ProblemFilters.exclude[FinalClassProblem]("akka.testkit.TestActor$UnWatch"),
ProblemFilters.exclude[FinalClassProblem]("akka.testkit.ErrorFilter"),
ProblemFilters.exclude[FinalClassProblem]("akka.testkit.InfoFilter"),
ProblemFilters.exclude[FinalClassProblem]("akka.testkit.TestActor$Watch"),
ProblemFilters.exclude[FinalClassProblem]("akka.testkit.WarningFilter"),
ProblemFilters.exclude[FinalClassProblem]("akka.testkit.DebugFilter"),
ProblemFilters.exclude[FinalClassProblem]("akka.testkit.DeadLettersFilter"),
ProblemFilters.exclude[FinalClassProblem]("akka.testkit.TestActor$RealMessage"),
ProblemFilters.exclude[FinalClassProblem]("akka.testkit.TestEvent$UnMute"),
ProblemFilters.exclude[FinalClassProblem]("akka.testkit.TestActor$SetIgnore"),
ProblemFilters.exclude[FinalClassProblem]("akka.testkit.TestActor$SetAutoPilot"),
ProblemFilters.exclude[FinalClassProblem]("akka.testkit.CustomEventFilter"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.EndpointManager$ListensFailure"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.InvalidAssociation"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.RemoteScope"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.AckedReceiveBuffer"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.RemoteWatcher$WatchRemote"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.EndpointManager$Gated"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.AckedSendBuffer"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.EndpointManager$Link"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.QuarantinedEvent"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.EndpointManager$ManagementCommand"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.EndpointManager$Send"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.Ack"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.RemoteWatcher$Stats"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.EndpointManager$Quarantined"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.RARP"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.EndpointManager$ResendState"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.RemoteWatcher$RewatchRemote"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.Remoting$RegisterTransportActor"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.ShutDownAssociation"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.EndpointManager$Quarantine"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.RemoteWatcher$UnwatchRemote"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.EndpointWriter$Handle"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.RemoteActorRefProvider$Internals"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.EndpointManager$ListensResult"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.RemoteWatcher$ExpectedFirstHeartbeat"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.RemoteDeploymentWatcher$WatchRemote"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.DaemonMsgCreate"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.RemoteWatcher$HeartbeatRsp"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.EndpointWriter$StoppedReading"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.EndpointManager$Listen"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.EndpointWriter$OutboundAck"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.HeartbeatHistory"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.HopelessAssociation"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.EndpointWriter$TookOver"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.SeqNo"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.ReliableDeliverySupervisor$GotUid"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.EndpointManager$ManagementCommandAck"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.FailureInjectorTransportAdapter$Drop"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.ThrottlerManager$AssociateResult"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.ThrottlerManager$Listener"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.ProtocolStateActor$OutboundUnassociated"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.ProtocolStateActor$Handle"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.ThrottlerTransportAdapter$SetThrottle"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.ProtocolStateActor$ListenerReady"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.ActorTransportAdapter$ListenUnderlying"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.ActorTransportAdapter$AssociateUnderlying"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.FailureInjectorTransportAdapter$One"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.TestTransport$DisassociateAttempt"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.ProtocolStateActor$HandleListenerRegistered"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.ProtocolStateActor$OutboundUnderlyingAssociated"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.ProtocolStateActor$InboundUnassociated"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.AkkaProtocolTransport$AssociateUnderlyingRefuseUid"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.TestTransport$AssociateAttempt"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.TestTransport$ListenAttempt"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.AkkaPduCodec$Disassociate"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.AkkaPduCodec$Message"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.Transport$InboundAssociation"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.Transport$InvalidAssociationException"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.ProtocolStateActor$AssociatedWaitHandler"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.TestAssociationHandle"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.ThrottlerManager$ListenerAndMode"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.AssociationHandle$Disassociated"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.AssociationHandle$InboundPayload"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.ThrottledAssociation$FailWith"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.Transport$ActorAssociationEventListener"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.FailureInjectorHandle"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.AkkaPduCodec$Payload"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.ThrottledAssociation$ExposedHandle"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.FailureInjectorTransportAdapter$All"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.TestTransport$WriteAttempt"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.ThrottlerManager$Handle"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.ThrottlerTransportAdapter$TokenBucket"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.ThrottlerManager$Checkin"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.HandshakeInfo"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.ThrottlerTransportAdapter$ForceDisassociate"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.ActorTransportAdapter$DisassociateUnderlying"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.FailureInjectorException"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.ThrottlerHandle"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.TestTransport$ShutdownAttempt"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.ActorTransportAdapter$ListenerRegistered"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.AkkaPduCodec$Associate"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.ThrottlerTransportAdapter$ForceDisassociateExplicitly"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.transport.AssociationHandle$ActorHandleEventListener"),
ProblemFilters.exclude[FinalClassProblem]("akka.camel.FailureResult"),
ProblemFilters.exclude[FinalClassProblem]("akka.camel.MessageResult"),
ProblemFilters.exclude[FinalClassProblem]("akka.camel.internal.CamelSupervisor$AddWatch"),
ProblemFilters.exclude[FinalClassProblem]("akka.camel.internal.AwaitDeActivation"),
ProblemFilters.exclude[FinalClassProblem]("akka.camel.internal.CamelSupervisor$CamelProducerObjects"),
ProblemFilters.exclude[FinalClassProblem]("akka.camel.internal.CamelSupervisor$DeRegister"),
ProblemFilters.exclude[FinalClassProblem]("akka.camel.internal.AwaitActivation"),
ProblemFilters.exclude[FinalClassProblem]("akka.camel.internal.CamelSupervisor$Register"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.ClientFSM$Data"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.ToClient"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.BarrierCoordinator$ClientLost"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.EnterBarrier"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.Remove"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.ToServer"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.DisconnectMsg"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.BarrierCoordinator$BarrierEmpty"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.Disconnect"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.ClientFSM$ConnectionFailure"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.RoleName"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.Hello"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.Controller$NodeInfo"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.Controller$CreateServerFSM"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.GetAddress"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.BarrierCoordinator$WrongBarrier"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.AddressReply"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.ClientFSM$Connected"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.BarrierCoordinator$RemoveClient"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.BarrierCoordinator$FailedBarrier"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.ThrottleMsg"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.BarrierCoordinator$Data"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.BarrierResult"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.BarrierCoordinator$BarrierTimeout"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.Controller$ClientDisconnected"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.FailBarrier"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.BarrierCoordinator$DuplicateNode"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testconductor.Throttle"),
ProblemFilters.exclude[FinalClassProblem]("akka.remote.testkit.MultiNodeSpec$Replacement"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.InternalClusterAction$Subscribe"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.ClusterEvent$SeenChanged"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.VectorClock"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.InternalClusterAction$PublishChanges"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.Metric"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.ClusterEvent$ReachableMember"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.ClusterUserAction$Down"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.ClusterHeartbeatSender$ExpectedFirstHeartbeat"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.InternalClusterAction$PublishEvent"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.ClusterEvent$RoleLeaderChanged"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.ClusterHeartbeatSender$HeartbeatRsp"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.ClusterEvent$ClusterMetricsChanged"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.AutoDown$UnreachableTimeout"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.ClusterEvent$CurrentInternalStats"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.ClusterEvent$MemberUp"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.ClusterEvent$CurrentClusterState"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.GossipOverview"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.GossipStatus"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.GossipStats"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.MetricsGossip"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.InternalClusterAction$Join"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.UniqueAddress"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.InternalClusterAction$SendGossipTo"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.InternalClusterAction$PublisherCreated"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.ClusterUserAction$Leave"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.Gossip"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.ClusterEvent$ReachabilityChanged"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.ClusterEvent$UnreachableMember"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.InternalClusterAction$AddOnMemberUpListener"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.ClusterEvent$LeaderChanged"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.InternalClusterAction$JoinSeedNodes"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.StandardMetrics$HeapMemory"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.VectorClockStats"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.NodeMetrics"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.Reachability$Record"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.InternalClusterAction$InitJoinAck"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.StandardMetrics$Cpu"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.InternalClusterAction$InitJoinNack"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.EWMA"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.InternalClusterAction$Unsubscribe"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.ClusterHeartbeatSender$Heartbeat"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.MetricsGossipEnvelope"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.ClusterEvent$MemberRemoved"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.ClusterUserAction$JoinTo"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.ClusterEvent$MemberExited"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.InternalClusterAction$Welcome"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.routing.ClusterRouterPoolSettings"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.routing.MixMetricsSelector"),
ProblemFilters.exclude[FinalClassProblem]("akka.cluster.routing.ClusterRouterGroupSettings"),
// changed to static method, source compatible is enough
ProblemFilters.exclude[MissingMethodProblem]("akka.testkit.JavaTestKit.shutdownActorSystem"),
// testActorName()java.lang.String in trait akka.testkit.TestKitBase does not have a correspondent in old version
ProblemFilters.exclude[MissingMethodProblem]("akka.testkit.TestKitBase.testActorName"),
// method remainingOrDefault()scala.concurrent.duration.FiniteDuration in trait akka.testkit.TestKitBase does not have a correspondent in old version
ProblemFilters.exclude[MissingMethodProblem]("akka.testkit.TestKitBase.remainingOrDefault"),
// synthetic method akka$remote$testkit$MultiNodeSpec$Replacement$$$outer()akka.remote.testkit.MultiNodeSpec in class akka.remote.testkit.MultiNodeSpec#Replacement does not have a correspondent in new version
ProblemFilters.exclude[MissingMethodProblem]("akka.remote.testkit.MultiNodeSpec#Replacement.akka$remote$testkit$MultiNodeSpec$Replacement$$$outer"),
// method nrOfInstances(akka.actor.ActorSystem)Int in trait akka.routing.Pool does not have a correspondent in old version
// ok to exclude, since we don't call nrOfInstances(sys) for old implementations
ProblemFilters.exclude[MissingMethodProblem]("akka.routing.Pool.nrOfInstances"),
// removed deprecated
ProblemFilters.exclude[MissingClassProblem]("akka.actor.UntypedActorFactory"),
ProblemFilters.exclude[MissingMethodProblem]("akka.util.Timeout.longToTimeout"),
ProblemFilters.exclude[MissingMethodProblem]("akka.util.Timeout.intToTimeout"),
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.util.Timeout.apply"),
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.util.Timeout.this"),
FilterAnyProblem("akka.routing.ConsistentHashingRouter"),
ProblemFilters.exclude[MissingClassProblem]("akka.routing.SmallestMailboxRouter$"),
ProblemFilters.exclude[MissingClassProblem]("akka.routing.RouterRoutees$"),
ProblemFilters.exclude[MissingClassProblem]("akka.routing.ScatterGatherFirstCompletedRouter"),
ProblemFilters.exclude[MissingClassProblem]("akka.routing.CurrentRoutees$"),
ProblemFilters.exclude[MissingClassProblem]("akka.routing.CurrentRoutees"),
ProblemFilters.exclude[MissingClassProblem]("akka.routing.RouterRoutees"),
ProblemFilters.exclude[MissingClassProblem]("akka.routing.RandomRouter"),
// class akka.routing.CollectRouteeRefs does not have a correspondent in new version
ProblemFilters.exclude[MissingClassProblem]("akka.routing.CollectRouteeRefs"),
// class akka.routing.ConsistentActorRef does not have a correspondent in new version
ProblemFilters.exclude[MissingClassProblem]("akka.routing.ConsistentActorRef"),
// object akka.routing.ConsistentActorRef does not have a correspondent in new version
ProblemFilters.exclude[MissingClassProblem]("akka.routing.ConsistentActorRef$"),
// object akka.routing.RandomRouter does not have a correspondent in new version
ProblemFilters.exclude[MissingClassProblem]("akka.routing.RandomRouter$"),
// object akka.routing.BroadcastRouter does not have a correspondent in new version
ProblemFilters.exclude[MissingClassProblem]("akka.routing.BroadcastRouter$"),
// class akka.routing.RoundRobinRouter does not have a correspondent in new version
ProblemFilters.exclude[MissingClassProblem]("akka.routing.RoundRobinRouter"),
// class akka.routing.BroadcastRouter does not have a correspondent in new version
ProblemFilters.exclude[MissingClassProblem]("akka.routing.BroadcastRouter"),
// class akka.routing.SmallestMailboxRouter does not have a correspondent in new version
ProblemFilters.exclude[MissingClassProblem]("akka.routing.SmallestMailboxRouter"),
// object akka.routing.ScatterGatherFirstCompletedRouter does not have a correspondent in new version
ProblemFilters.exclude[MissingClassProblem]("akka.routing.ScatterGatherFirstCompletedRouter$"),
// interface akka.routing.DeprecatedRouterConfig does not have a correspondent in new version
ProblemFilters.exclude[MissingClassProblem]("akka.routing.DeprecatedRouterConfig"),
// object akka.routing.RoundRobinRouter does not have a correspondent in new version
ProblemFilters.exclude[MissingClassProblem]("akka.routing.RoundRobinRouter$"),
// method toString()java.lang.String in object akka.routing.BalancingPool does not have a correspondent in new version
ProblemFilters.exclude[MissingMethodProblem]("akka.routing.BalancingPool.toString"),
ProblemFilters.exclude[MissingMethodProblem]("akka.remote.RemoteSettings.LogRemoteLifecycleEvents"),
ProblemFilters.exclude[MissingMethodProblem]("akka.cluster.Cluster.publishCurrentClusterState"),
ProblemFilters.exclude[MissingClassProblem]("akka.cluster.InternalClusterAction$PublishCurrentClusterState$"),
ProblemFilters.exclude[MissingMethodProblem]("akka.cluster.ClusterSettings.AutoDown"),
// class akka.cluster.routing.ClusterRouterSettings does not have a correspondent in new version
ProblemFilters.exclude[MissingClassProblem]("akka.cluster.routing.ClusterRouterSettings"),
// object akka.cluster.routing.ClusterRouterConfig does not have a correspondent in new version
ProblemFilters.exclude[MissingClassProblem]("akka.cluster.routing.ClusterRouterConfig$"),
// object akka.cluster.routing.AdaptiveLoadBalancingRouter does not have a correspondent in new version
ProblemFilters.exclude[MissingClassProblem]("akka.cluster.routing.AdaptiveLoadBalancingRouter$"),
// object akka.cluster.routing.ClusterRouterSettings does not have a correspondent in new version
ProblemFilters.exclude[MissingClassProblem]("akka.cluster.routing.ClusterRouterSettings$"),
// class akka.cluster.routing.AdaptiveLoadBalancingRouter does not have a correspondent in new version
ProblemFilters.exclude[MissingClassProblem]("akka.cluster.routing.AdaptiveLoadBalancingRouter"),
// class akka.cluster.routing.ClusterRouterConfig does not have a correspondent in new version
ProblemFilters.exclude[MissingClassProblem]("akka.cluster.routing.ClusterRouterConfig"),
// deprecated method this(Int,java.lang.String,Boolean,java.lang.String)Unit in class akka.cluster.routing.ClusterRouterGroupSettings does not have a correspondent with same parameter signature among (Int,java.lang.Iterable,Boolean,java.lang.String)Unit, (Int,scala.collection.immutable.Seq,Boolean,scala.Option)Unit
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.cluster.routing.ClusterRouterGroupSettings.this"),
// deprecated method this(Int,java.lang.String,Boolean,scala.Option)Unit in class akka.cluster.routing.ClusterRouterGroupSettings does not have a correspondent with same parameter signature among (Int,java.lang.Iterable,Boolean,java.lang.String)Unit, (Int,scala.collection.immutable.Seq,Boolean,scala.Option)Unit
ProblemFilters.exclude[IncompatibleMethTypeProblem]("akka.cluster.routing.ClusterRouterGroupSettings.this"),
ProblemFilters.exclude[MissingMethodProblem]("akka.testkit.TestKit.dilated"),
// changed internals
ProblemFilters.exclude[MissingMethodProblem]("akka.actor.ActorSystem.terminate"),
ProblemFilters.exclude[MissingMethodProblem]("akka.actor.ActorSystem.whenTerminated"),
ProblemFilters.exclude[MissingMethodProblem]("akka.actor.ExtendedActorSystem.logFilter"),
// the type hierarchy of class akka.actor.ChildActorPath has changed in new version. Missing types {akka.actor.ActorPath}
ProblemFilters.exclude[MissingTypesProblem]("akka.actor.ChildActorPath"),
// the type hierarchy of class akka.actor.RootActorPath has changed in new version. Missing types {akka.actor.ActorPath}
ProblemFilters.exclude[MissingTypesProblem]("akka.actor.RootActorPath"),
// declaration of trait akka.actor.ActorPath has changed to class akka.actor.ActorPath in new version; changing trait to class breaks client code
ProblemFilters.exclude[IncompatibleTemplateDefProblem]("akka.actor.ActorPath"),
ProblemFilters.exclude[MissingMethodProblem]("akka.actor.ActorPath.ValidSymbols"),
ProblemFilters.exclude[MissingMethodProblem]("akka.actor.LocalActorRefProvider.terminationPromise"),
ProblemFilters.exclude[MissingClassProblem]("akka.actor.UntypedActorFactoryConsumer"),
ProblemFilters.exclude[MissingMethodProblem]("akka.actor.ActorSystemImpl.terminationFuture"),
ProblemFilters.exclude[MissingMethodProblem]("akka.actor.IndirectActorProducer.UntypedActorFactoryConsumerClass"),
FilterAnyProblem("akka.actor.ActorSystemImpl"),
ProblemFilters.exclude[MissingMethodProblem]("akka.pattern.AskSupport.ask"),
FilterAnyProblem("akka.actor.ActorSystemImpl$TerminationCallbacks"),
ProblemFilters.exclude[MissingMethodProblem]("akka.event.Logging#LogEvent.getMDC"),
ProblemFilters.exclude[MissingMethodProblem]("akka.util.ByteString.byteStringCompanion"),
ProblemFilters.exclude[MissingMethodProblem]("akka.util.ByteString.writeToOutputStream"),
//method boss()akka.actor.RepointableActorRef in class akka.actor.ActorDSL#Extension does not have a correspondent in new version
ProblemFilters.exclude[MissingMethodProblem]("akka.actor.ActorDSL#Extension.boss"),
// method hasSubscriptions(java.lang.Object)Boolean in trait akka.event.SubchannelClassification does not have a correspondent in old version
// ok to exclude since it is only invoked from new EventStreamUnsubscriber
ProblemFilters.exclude[MissingMethodProblem]("akka.event.SubchannelClassification.hasSubscriptions"),
FilterAnyProblem("akka.remote.EndpointManager"),
FilterAnyProblem("akka.remote.RemoteTransport"),
FilterAnyProblem("akka.remote.Remoting"),
FilterAnyProblem("akka.remote.PhiAccrualFailureDetector$State"),
FilterAnyProblem("akka.cluster.ClusterDomainEventPublisher"),
FilterAnyProblem("akka.cluster.InternalClusterAction"),
ProblemFilters.exclude[MissingClassProblem]("akka.cluster.InternalClusterAction$PublishCurrentClusterState")
) )
} }
} }