= #17342 Make 2.4 binary compatible with 2.3
(cherry picked from commit89af8bdb90) * 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 seea6d3704ef6* 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:
parent
412491d277
commit
b30e460be7
50 changed files with 1037 additions and 202 deletions
|
|
@ -147,7 +147,7 @@ abstract class EventBusSpec(busName: String, conf: Config = ConfigFactory.empty(
|
|||
|
||||
object ActorEventBusSpec {
|
||||
class MyActorEventBus(protected val system: ActorSystem) extends ActorEventBus
|
||||
with ActorClassification with ActorClassifier {
|
||||
with ManagedActorClassification with ActorClassifier {
|
||||
|
||||
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)
|
||||
|
||||
// ActorClassification specific tests
|
||||
// ManagedActorClassification specific tests
|
||||
|
||||
"must unsubscribe subscriber when it terminates" in {
|
||||
val a1 = createSubscriber(system.deadLetters)
|
||||
|
|
|
|||
|
|
@ -11,6 +11,7 @@ import akka.actor.ActorRef
|
|||
import akka.io.Udp._
|
||||
import akka.io.Inet._
|
||||
import akka.testkit.SocketUtil._
|
||||
import java.net.DatagramSocket
|
||||
|
||||
class UdpIntegrationSpec extends AkkaSpec("""
|
||||
akka.loglevel = INFO
|
||||
|
|
@ -86,7 +87,7 @@ class UdpIntegrationSpec extends AkkaSpec("""
|
|||
|
||||
"call SocketOption.afterConnect method after binding." in {
|
||||
val commander = TestProbe()
|
||||
val assertOption = AssertAfterConnect()
|
||||
val assertOption = AssertAfterChannelBind()
|
||||
commander.send(IO(Udp), Bind(testActor, addresses(4), options = List(assertOption)))
|
||||
commander.expectMsg(Bound(addresses(4)))
|
||||
assert(assertOption.afterCalled === 1)
|
||||
|
|
@ -106,17 +107,17 @@ class UdpIntegrationSpec extends AkkaSpec("""
|
|||
private case class AssertBeforeBind() extends SocketOption {
|
||||
var beforeCalled = 0
|
||||
|
||||
override def beforeBind(c: DatagramChannel) = {
|
||||
assert(!c.socket.isBound)
|
||||
override def beforeDatagramBind(ds: DatagramSocket): Unit = {
|
||||
assert(!ds.isBound)
|
||||
beforeCalled += 1
|
||||
}
|
||||
}
|
||||
|
||||
private case class AssertAfterConnect() extends SocketOption {
|
||||
private case class AssertAfterChannelBind() extends SocketOptionV2 {
|
||||
var afterCalled = 0
|
||||
|
||||
override def afterConnect(c: DatagramChannel) = {
|
||||
assert(c.socket.isBound)
|
||||
override def afterBind(s: DatagramSocket) = {
|
||||
assert(s.isBound)
|
||||
afterCalled += 1
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -17,6 +17,9 @@ object ActorPath {
|
|||
case _ ⇒ throw new MalformedURLException("cannot parse as ActorPath: " + s)
|
||||
}
|
||||
|
||||
@deprecated("Use `isValidPathElement` instead", since = "2.3.8")
|
||||
val ElementRegex = """(?:[-\w:@&=+,.!~*'_;]|%\p{XDigit}{2})(?:[-\w:@&=+,.!~*'$_;]|%\p{XDigit}{2})*""".r
|
||||
|
||||
private final val ValidSymbols = """-_.*$+:@&=,!~';"""
|
||||
|
||||
private final val ValidPathCode = -1
|
||||
|
|
|
|||
|
|
@ -122,18 +122,38 @@ object FSM {
|
|||
/** Used by `forMax` to signal "cancel stateTimeout" */
|
||||
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
|
||||
* name, the state data, possibly custom timeout, stop reason and replies
|
||||
* 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] = {
|
||||
State(stateName, stateData, timeout, stopReason, replies)(notifies)
|
||||
private[akka] def notifies: Boolean = true
|
||||
|
||||
// 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))
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
*/
|
||||
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
|
||||
*/
|
||||
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.
|
||||
|
|
@ -341,7 +367,7 @@ trait FSM[S, D] extends Actor with Listeners with ActorLogging {
|
|||
* @param nextStateName state designator for the next state
|
||||
* @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.
|
||||
|
|
|
|||
|
|
@ -180,6 +180,8 @@ trait SubchannelClassification { this: EventBus ⇒
|
|||
* Expensive call! Avoid calling directly from event bus subscribe / unsubscribe.
|
||||
*/
|
||||
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 }
|
||||
|
||||
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.
|
||||
* 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
|
||||
|
||||
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 add(monitored: ActorRef, monitor: ActorRef) = {
|
||||
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) = {
|
||||
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) = {
|
||||
val v = backing - monitored
|
||||
new ActorClassificationMappings(seqNr + 1, v)
|
||||
new ManagedActorClassificationMappings(seqNr + 1, v)
|
||||
}
|
||||
}
|
||||
|
||||
private val mappings = new AtomicReference[ActorClassificationMappings](
|
||||
new ActorClassificationMappings(0, Map.empty[ActorRef, immutable.TreeSet[ActorRef]]))
|
||||
private val mappings = new AtomicReference[ManagedActorClassificationMappings](
|
||||
new ManagedActorClassificationMappings(0, Map.empty[ActorRef, immutable.TreeSet[ActorRef]]))
|
||||
|
||||
private val empty = immutable.TreeSet.empty[ActorRef]
|
||||
|
||||
|
|
@ -409,3 +411,113 @@ trait ActorClassification { this: ActorEventBus with ActorClassifier ⇒
|
|||
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)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
*/
|
||||
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._
|
||||
|
||||
|
|
@ -130,13 +130,13 @@ private[akka] object ActorClassificationUnsubscriber {
|
|||
final case class Register(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")
|
||||
system.asInstanceOf[ExtendedActorSystem]
|
||||
.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)
|
||||
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -11,6 +11,11 @@ import akka.util.Subclassification
|
|||
import java.util.concurrent.atomic.AtomicReference
|
||||
import scala.annotation.tailrec
|
||||
|
||||
object EventStream {
|
||||
@deprecated("Use explicit `system.eventStream` instead", "2.4")
|
||||
implicit def fromActorSystem(system: ActorSystem) = system.eventStream
|
||||
}
|
||||
|
||||
/**
|
||||
* An Akka EventStream is a pub-sub stream of events both system and user generated,
|
||||
* where subscribers are ActorRefs and the channels are Classes and Events are any java.lang.Object.
|
||||
|
|
@ -20,7 +25,12 @@ import scala.annotation.tailrec
|
|||
* The debug flag in the constructor toggles if operations on this EventStream should also be published
|
||||
* 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 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 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 = {
|
||||
|
|
@ -65,14 +76,18 @@ class EventStream(sys: ActorSystem, private val debug: Boolean = false) extends
|
|||
* ''Must'' be called after actor system is "ready".
|
||||
* 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
|
||||
*/
|
||||
@tailrec
|
||||
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) ⇒
|
||||
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"))
|
||||
|
|
@ -95,7 +110,8 @@ class EventStream(sys: ActorSystem, private val debug: Boolean = false) extends
|
|||
*/
|
||||
@tailrec
|
||||
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) ⇒
|
||||
if (!initiallySubscribedOrUnsubscriber.compareAndSet(value, Left(subscribers + subscriber)))
|
||||
registerWithUnsubscriber(subscriber)
|
||||
|
|
@ -114,7 +130,8 @@ class EventStream(sys: ActorSystem, private val debug: Boolean = false) extends
|
|||
*/
|
||||
@tailrec
|
||||
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) ⇒
|
||||
if (!initiallySubscribedOrUnsubscriber.compareAndSet(value, Left(subscribers - subscriber)))
|
||||
unregisterIfNoMoreSubscribedChannels(subscriber)
|
||||
|
|
|
|||
|
|
@ -183,18 +183,51 @@ abstract class ScanningEventBus[E, S, C] extends EventBus[E, S, C] {
|
|||
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]]
|
||||
* 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 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 {
|
||||
type Event = E
|
||||
|
||||
override val system = ActorEventBus.this.system
|
||||
|
||||
override protected def mapSize: Int = ActorEventBus.this.mapSize
|
||||
|
||||
override protected def classify(event: E): ActorRef =
|
||||
|
|
|
|||
|
|
@ -4,6 +4,9 @@
|
|||
package akka.io
|
||||
|
||||
import java.nio.channels.{ DatagramChannel, SocketChannel, ServerSocketChannel }
|
||||
import java.net.DatagramSocket
|
||||
import java.net.ServerSocket
|
||||
import java.net.Socket
|
||||
|
||||
object Inet {
|
||||
|
||||
|
|
@ -16,37 +19,57 @@ object Inet {
|
|||
/**
|
||||
* 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
|
||||
*/
|
||||
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
|
||||
* the slave socket for servers).
|
||||
*/
|
||||
def afterConnect(c: DatagramChannel): 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 = ()
|
||||
def afterConnect(s: Socket): 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.
|
||||
*/
|
||||
|
|
@ -76,9 +99,9 @@ object Inet {
|
|||
*/
|
||||
final case class ReceiveBufferSize(size: Int) extends SocketOption {
|
||||
require(size > 0, "ReceiveBufferSize must be > 0")
|
||||
override def beforeBind(c: ServerSocketChannel): Unit = c.socket.setReceiveBufferSize(size)
|
||||
override def beforeBind(c: DatagramChannel): Unit = c.socket.setReceiveBufferSize(size)
|
||||
override def beforeBind(c: SocketChannel): Unit = c.socket.setReceiveBufferSize(size)
|
||||
override def beforeServerSocketBind(s: ServerSocket): Unit = s.setReceiveBufferSize(size)
|
||||
override def beforeDatagramBind(s: DatagramSocket): Unit = s.setReceiveBufferSize(size)
|
||||
override def beforeConnect(s: Socket): Unit = s.setReceiveBufferSize(size)
|
||||
}
|
||||
|
||||
// server socket options
|
||||
|
|
@ -89,9 +112,9 @@ object Inet {
|
|||
* For more information see [[java.net.Socket.setReuseAddress]]
|
||||
*/
|
||||
final case class ReuseAddress(on: Boolean) extends SocketOption {
|
||||
override def beforeBind(c: ServerSocketChannel): Unit = c.socket.setReuseAddress(on)
|
||||
override def beforeBind(c: DatagramChannel): Unit = c.socket.setReuseAddress(on)
|
||||
override def beforeBind(c: SocketChannel): Unit = c.socket.setReuseAddress(on)
|
||||
override def beforeServerSocketBind(s: ServerSocket): Unit = s.setReuseAddress(on)
|
||||
override def beforeDatagramBind(s: DatagramSocket): Unit = s.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 {
|
||||
require(size > 0, "SendBufferSize must be > 0")
|
||||
override def afterConnect(c: DatagramChannel): Unit = c.socket.setSendBufferSize(size)
|
||||
override def afterConnect(c: SocketChannel): Unit = c.socket.setSendBufferSize(size)
|
||||
override def afterConnect(s: Socket): Unit = s.setSendBufferSize(size)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -114,8 +136,7 @@ object Inet {
|
|||
*/
|
||||
final case class TrafficClass(tc: Int) extends SocketOption {
|
||||
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(c: SocketChannel): Unit = c.socket.setTrafficClass(tc)
|
||||
override def afterConnect(s: Socket): Unit = s.setTrafficClass(tc)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -185,43 +206,4 @@ object Inet {
|
|||
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 = ()
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -7,7 +7,6 @@ package akka.io
|
|||
import java.net.InetSocketAddress
|
||||
import java.net.Socket
|
||||
import akka.ConfigurationException
|
||||
import java.nio.channels.SocketChannel
|
||||
import akka.io.Inet._
|
||||
import com.typesafe.config.Config
|
||||
import scala.concurrent.duration._
|
||||
|
|
@ -58,7 +57,7 @@ object Tcp extends ExtensionId[TcpExt] with ExtensionIdProvider {
|
|||
* For more information see [[java.net.Socket.setKeepAlive]]
|
||||
*/
|
||||
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]]
|
||||
*/
|
||||
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
|
||||
|
|
@ -83,7 +82,7 @@ object Tcp extends ExtensionId[TcpExt] with ExtensionIdProvider {
|
|||
* For more information see [[java.net.Socket.setTcpNoDelay]]
|
||||
*/
|
||||
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)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -187,7 +187,7 @@ private[io] abstract class TcpConnection(val tcp: TcpExt, val channel: SocketCha
|
|||
// continue anyway
|
||||
log.debug("Could not enable TcpNoDelay: {}", e.getMessage)
|
||||
}
|
||||
options.foreach(_.afterConnect(channel))
|
||||
options.foreach(_.afterConnect(channel.socket))
|
||||
|
||||
commander ! Connected(
|
||||
channel.socket.getRemoteSocketAddress.asInstanceOf[InetSocketAddress],
|
||||
|
|
|
|||
|
|
@ -49,7 +49,7 @@ private[io] class TcpListener(selectorRouter: ActorRef,
|
|||
val localAddress =
|
||||
try {
|
||||
val socket = channel.socket
|
||||
bind.options.foreach(_.beforeBind(channel))
|
||||
bind.options.foreach(_.beforeServerSocketBind(socket))
|
||||
socket.bind(bind.localAddress, bind.backlog)
|
||||
val ret = socket.getLocalSocketAddress match {
|
||||
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)
|
||||
log.debug("Successfully bound to {}", ret)
|
||||
bind.options.foreach(_.afterConnect(channel))
|
||||
bind.options.foreach {
|
||||
case o: Inet.SocketOptionV2 ⇒ o.afterBind(channel.socket)
|
||||
case _ ⇒
|
||||
}
|
||||
ret
|
||||
} catch {
|
||||
case NonFatal(e) ⇒
|
||||
|
|
|
|||
|
|
@ -32,7 +32,7 @@ private[io] class TcpOutgoingConnection(_tcp: TcpExt,
|
|||
|
||||
context.watch(commander) // sign death pact
|
||||
|
||||
options.foreach(_.beforeBind(channel))
|
||||
options.foreach(_.beforeConnect(channel.socket))
|
||||
localAddress.foreach(channel.socket.bind)
|
||||
channelRegistry.register(channel, 0)
|
||||
timeout foreach context.setReceiveTimeout //Initiate connection timeout if supplied
|
||||
|
|
|
|||
|
|
@ -3,8 +3,8 @@
|
|||
*/
|
||||
package akka.io
|
||||
|
||||
import java.net.DatagramSocket
|
||||
import java.net.InetSocketAddress
|
||||
import java.nio.channels.DatagramChannel
|
||||
import com.typesafe.config.Config
|
||||
import scala.collection.immutable
|
||||
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]]
|
||||
*/
|
||||
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)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -57,7 +57,7 @@ private[io] class UdpConnection(udpConn: UdpConnectedExt,
|
|||
channel = DatagramChannel.open
|
||||
channel.configureBlocking(false)
|
||||
val socket = channel.socket
|
||||
options.foreach(_.beforeBind(channel))
|
||||
options.foreach(_.beforeDatagramBind(socket))
|
||||
localAddress foreach socket.bind
|
||||
channel.connect(remoteAddress)
|
||||
channelRegistry.register(channel, OP_READ)
|
||||
|
|
@ -67,7 +67,10 @@ private[io] class UdpConnection(udpConn: UdpConnectedExt,
|
|||
|
||||
def receive = {
|
||||
case registration: ChannelRegistration ⇒
|
||||
options.foreach(_.afterConnect(channel))
|
||||
options.foreach {
|
||||
case v2: Inet.SocketOptionV2 ⇒ v2.afterConnect(channel.socket)
|
||||
case _ ⇒
|
||||
}
|
||||
commander ! Connected
|
||||
context.become(connected(registration), discardOld = true)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -40,7 +40,7 @@ private[io] class UdpListener(val udp: UdpExt,
|
|||
val localAddress =
|
||||
try {
|
||||
val socket = channel.socket
|
||||
bind.options.foreach(_.beforeBind(channel))
|
||||
bind.options.foreach(_.beforeDatagramBind(socket))
|
||||
socket.bind(bind.localAddress)
|
||||
val ret = socket.getLocalSocketAddress match {
|
||||
case isa: InetSocketAddress ⇒ isa
|
||||
|
|
@ -48,7 +48,10 @@ private[io] class UdpListener(val udp: UdpExt,
|
|||
}
|
||||
channelRegistry.register(channel, OP_READ)
|
||||
log.debug("Successfully bound to [{}]", ret)
|
||||
bind.options.foreach(_.afterConnect(channel))
|
||||
bind.options.foreach {
|
||||
case o: Inet.SocketOptionV2 ⇒ o.afterBind(channel.socket)
|
||||
case _ ⇒
|
||||
}
|
||||
ret
|
||||
} catch {
|
||||
case NonFatal(e) ⇒
|
||||
|
|
|
|||
|
|
@ -23,7 +23,8 @@ private[io] class UdpSender(val udp: UdpExt,
|
|||
val channel = {
|
||||
val datagramChannel = DatagramChannel.open
|
||||
datagramChannel.configureBlocking(false)
|
||||
options foreach { _.beforeBind(datagramChannel) }
|
||||
val socket = datagramChannel.socket
|
||||
options foreach { _.beforeDatagramBind(socket) }
|
||||
|
||||
datagramChannel
|
||||
}
|
||||
|
|
@ -31,7 +32,10 @@ private[io] class UdpSender(val udp: UdpExt,
|
|||
|
||||
def receive: Receive = {
|
||||
case registration: ChannelRegistration ⇒
|
||||
options.foreach(_.afterConnect(channel))
|
||||
options.foreach {
|
||||
case v2: Inet.SocketOptionV2 ⇒ v2.afterConnect(channel.socket)
|
||||
case _ ⇒
|
||||
}
|
||||
commander ! SimpleSenderReady
|
||||
context.become(sendHandlers(registration))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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] =
|
||||
actorRef.?(message)(timeout, sender)
|
||||
actorRef.internalAsk(message, timeout, sender)
|
||||
|
||||
/**
|
||||
* 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] =
|
||||
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 {
|
||||
|
||||
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 ⇒
|
||||
actorRef ! message
|
||||
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}"."""))
|
||||
}
|
||||
|
||||
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 {
|
||||
|
||||
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 ⇒
|
||||
if (timeout.duration.length <= 0)
|
||||
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}"."""))
|
||||
}
|
||||
|
||||
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 PromiseActorRef._
|
||||
|
||||
@deprecated("Use the full constructor", "2.4")
|
||||
def this(provider: ActorRefProvider, result: Promise[Any]) = this(provider, result, "unknown")
|
||||
|
||||
// This is necessary for weaving the PromiseActorRef into the asked message, i.e. the replyTo pattern.
|
||||
@volatile var messageClassName = _mcn
|
||||
|
||||
|
|
@ -344,4 +416,8 @@ private[akka] object PromiseActorRef {
|
|||
result.future onComplete { _ ⇒ try a.stop() finally f.cancel() }
|
||||
a
|
||||
}
|
||||
|
||||
@deprecated("Use apply with messageClassName and sender parameters", "2.4")
|
||||
def apply(provider: ActorRefProvider, timeout: Timeout, targetName: String): PromiseActorRef =
|
||||
apply(provider, timeout, targetName, "unknown", Actor.noSender)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -66,7 +66,7 @@ private[akka] final class BalancingRoutingLogic extends RoutingLogic {
|
|||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class BalancingPool(
|
||||
nrOfInstances: Int,
|
||||
override val nrOfInstances: Int,
|
||||
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
||||
extends Pool {
|
||||
|
|
@ -101,7 +101,7 @@ final case class BalancingPool(
|
|||
override private[akka] def newRoutee(routeeProps: Props, context: ActorContext): Routee = {
|
||||
|
||||
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, '_')
|
||||
}
|
||||
val dispatcherId = s"BalancingPool-$deployPath"
|
||||
|
|
@ -113,7 +113,7 @@ final case class BalancingPool(
|
|||
val deployDispatcherConfigPath = s"akka.actor.deployment.$deployPath.pool-dispatcher"
|
||||
val systemConfig = context.system.settings.config
|
||||
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)
|
||||
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('$', '@', ':')
|
||||
}
|
||||
|
|
|
|||
|
|
@ -58,7 +58,7 @@ final class BroadcastRoutingLogic extends RoutingLogic {
|
|||
*/
|
||||
@SerialVersionUID(1L)
|
||||
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 routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
override val usePoolDispatcher: Boolean = false)
|
||||
|
|
|
|||
|
|
@ -260,7 +260,7 @@ final case class ConsistentHashingRoutingLogic(
|
|||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class ConsistentHashingPool(
|
||||
val nrOfInstances: Int,
|
||||
override val nrOfInstances: Int,
|
||||
override val resizer: Option[Resizer] = None,
|
||||
val virtualNodesFactor: Int = 0,
|
||||
val hashMapping: ConsistentHashingRouter.ConsistentHashMapping = ConsistentHashingRouter.emptyConsistentHashMapping,
|
||||
|
|
|
|||
|
|
@ -59,7 +59,7 @@ final class RandomRoutingLogic extends RoutingLogic {
|
|||
*/
|
||||
@SerialVersionUID(1L)
|
||||
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 routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
override val usePoolDispatcher: Boolean = false)
|
||||
|
|
|
|||
|
|
@ -64,7 +64,7 @@ final class RoundRobinRoutingLogic extends RoutingLogic {
|
|||
*/
|
||||
@SerialVersionUID(1L)
|
||||
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 routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
override val usePoolDispatcher: Boolean = false)
|
||||
|
|
|
|||
|
|
@ -102,7 +102,10 @@ private[akka] class RoutedActorCell(
|
|||
_router = routerConfig.createRouter(system)
|
||||
routerConfig match {
|
||||
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)
|
||||
addRoutees(Vector.fill(nrOfRoutees)(pool.newRoutee(routeeProps, this)))
|
||||
case group: Group ⇒
|
||||
|
|
|
|||
|
|
@ -177,6 +177,9 @@ abstract class PoolBase extends Pool
|
|||
*/
|
||||
trait Pool extends RouterConfig {
|
||||
|
||||
@deprecated("Implement nrOfInstances with ActorSystem parameter instead", "2.4")
|
||||
def nrOfInstances: Int = -1
|
||||
|
||||
/**
|
||||
* Initial number of routee instances
|
||||
*/
|
||||
|
|
|
|||
|
|
@ -93,7 +93,7 @@ private[akka] final case class ScatterGatherFirstCompletedRoutees(
|
|||
*/
|
||||
@SerialVersionUID(1L)
|
||||
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,
|
||||
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
|
|
|
|||
|
|
@ -174,7 +174,7 @@ class SmallestMailboxRoutingLogic extends RoutingLogic {
|
|||
*/
|
||||
@SerialVersionUID(1L)
|
||||
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 routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
override val usePoolDispatcher: Boolean = false)
|
||||
|
|
|
|||
|
|
@ -120,7 +120,7 @@ private[akka] final case class TailChoppingRoutees(
|
|||
*/
|
||||
@SerialVersionUID(1L)
|
||||
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,
|
||||
interval: FiniteDuration,
|
||||
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
||||
|
|
|
|||
|
|
@ -73,7 +73,7 @@ trait BaseSerializer extends Serializer {
|
|||
/**
|
||||
* Actor system which is required by most serializer implementations.
|
||||
*/
|
||||
val system: ExtendedActorSystem
|
||||
def system: ExtendedActorSystem
|
||||
/**
|
||||
* Configuration namespace of serialization identifiers in the `reference.conf`.
|
||||
*
|
||||
|
|
@ -88,7 +88,12 @@ trait BaseSerializer extends Serializer {
|
|||
*
|
||||
* 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}"""")
|
||||
}
|
||||
|
||||
|
|
@ -146,6 +151,14 @@ object JavaSerializer {
|
|||
*/
|
||||
class JavaSerializer(val system: ExtendedActorSystem) extends BaseSerializer {
|
||||
|
||||
@deprecated("Use constructor with ExtendedActorSystem", "2.4")
|
||||
def this() = this(null)
|
||||
|
||||
// TODO remove this when deprecated this() is removed
|
||||
override val identifier: Int =
|
||||
if (system eq null) 1
|
||||
else identifierFromConfig
|
||||
|
||||
def includeManifest: Boolean = false
|
||||
|
||||
def toBinary(o: AnyRef): Array[Byte] = {
|
||||
|
|
@ -180,6 +193,15 @@ class NullSerializer extends Serializer {
|
|||
* (just returns the byte array unchanged/uncopied)
|
||||
*/
|
||||
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 toBinary(o: AnyRef) = o match {
|
||||
case null ⇒ null
|
||||
|
|
|
|||
|
|
@ -121,7 +121,7 @@ final case class AdaptiveLoadBalancingRoutingLogic(system: ActorSystem, metricsS
|
|||
@SerialVersionUID(1L)
|
||||
final case class AdaptiveLoadBalancingPool(
|
||||
metricsSelector: MetricsSelector = MixMetricsSelector,
|
||||
val nrOfInstances: Int = 0,
|
||||
override val nrOfInstances: Int = 0,
|
||||
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
override val usePoolDispatcher: Boolean = false)
|
||||
|
|
|
|||
|
|
@ -24,6 +24,14 @@ import scala.concurrent.duration.Deadline
|
|||
*/
|
||||
class ClusterMessageSerializer(val system: ExtendedActorSystem) extends BaseSerializer {
|
||||
|
||||
@deprecated("Use constructor with ExtendedActorSystem", "2.4")
|
||||
def this() = this(null)
|
||||
|
||||
// TODO remove this when deprecated this() is removed
|
||||
override val identifier: Int =
|
||||
if (system eq null) 5
|
||||
else identifierFromConfig
|
||||
|
||||
private final val BufferSize = 1024 * 4
|
||||
// must be lazy because serializer is initialized from Cluster extension constructor
|
||||
private lazy val GossipTimeToLive = Cluster(system).settings.GossipTimeToLive
|
||||
|
|
|
|||
|
|
@ -132,7 +132,7 @@ final case class AdaptiveLoadBalancingRoutingLogic(system: ActorSystem, metricsS
|
|||
@deprecated("Superseded by akka.cluster.metrics (in akka-cluster-metrics jar)", "2.4")
|
||||
final case class AdaptiveLoadBalancingPool(
|
||||
metricsSelector: MetricsSelector = MixMetricsSelector,
|
||||
val nrOfInstances: Int = 0,
|
||||
override val nrOfInstances: Int = 0,
|
||||
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
override val usePoolDispatcher: Boolean = false)
|
||||
|
|
|
|||
|
|
@ -39,7 +39,7 @@ import akka.event.japi.ScanningEventBus;
|
|||
//#scanning-bus
|
||||
|
||||
//#actor-bus
|
||||
import akka.event.japi.ActorEventBus;
|
||||
import akka.event.japi.ManagedActorEventBus;
|
||||
|
||||
//#actor-bus
|
||||
|
||||
|
|
@ -232,7 +232,7 @@ public class EventBusDocTest {
|
|||
|
||||
static
|
||||
//#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
|
||||
public ActorBusImpl(ActorSystem system) {
|
||||
|
|
@ -304,7 +304,7 @@ public class EventBusDocTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateActorClassification() {
|
||||
public void demonstrateManagedActorClassification() {
|
||||
//#actor-bus-test
|
||||
ActorRef observer1 = new JavaTestKit(system).getRef();
|
||||
ActorRef observer2 = new JavaTestKit(system).getRef();
|
||||
|
|
|
|||
|
|
@ -115,7 +115,7 @@ type :class:`ActorRef`.
|
|||
|
||||
This classification requires an :class:`ActorSystem` in order to perform book-keeping
|
||||
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.
|
||||
|
||||
The necessary methods to be implemented are illustrated with the following example:
|
||||
|
|
|
|||
|
|
@ -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>`
|
||||
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
|
||||
========================================
|
||||
|
||||
|
|
@ -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
|
||||
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
|
||||
``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
|
||||
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:
|
||||
|
||||
|
|
@ -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.
|
||||
|
||||
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
|
||||
==================================
|
||||
Previously in ``2.3.x`` entries were direct children of the local ``ShardRegion``. In examples the ``persistenceId`` of entries
|
||||
|
|
|
|||
|
|
@ -121,12 +121,12 @@ object EventBusDocSpec {
|
|||
|
||||
//#actor-bus
|
||||
import akka.event.ActorEventBus
|
||||
import akka.event.ActorClassification
|
||||
import akka.event.ManagedActorClassification
|
||||
import akka.event.ActorClassifier
|
||||
|
||||
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
|
||||
|
||||
// is used for extracting the classifier from the incoming events
|
||||
|
|
@ -179,7 +179,7 @@ class EventBusDocSpec extends AkkaSpec {
|
|||
//#scanning-bus-test
|
||||
}
|
||||
|
||||
"demonstrate ActorClassification" in {
|
||||
"demonstrate ManagedActorClassification" in {
|
||||
//#actor-bus-test
|
||||
val observer1 = TestProbe().ref
|
||||
val observer2 = TestProbe().ref
|
||||
|
|
|
|||
|
|
@ -115,7 +115,7 @@ type :class:`ActorRef`.
|
|||
|
||||
This classification requires an :class:`ActorSystem` in order to perform book-keeping
|
||||
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.
|
||||
|
||||
The necessary methods to be implemented are illustrated with the following example:
|
||||
|
|
|
|||
|
|
@ -231,7 +231,10 @@ private[remote] class Remoting(_system: ExtendedActorSystem, _provider: RemoteAc
|
|||
private[akka] def boundAddresses: Map[String, Set[Address]] = {
|
||||
transportMapping.map {
|
||||
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)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -29,6 +29,14 @@ private[akka] class DaemonMsgCreateSerializer(val system: ExtendedActorSystem) e
|
|||
import ProtobufSerializer.deserializeActorRef
|
||||
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
|
||||
|
||||
lazy val serialization = SerializationExtension(system)
|
||||
|
|
|
|||
|
|
@ -17,6 +17,14 @@ import akka.serialization.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 toBinary(obj: AnyRef): Array[Byte] = obj match {
|
||||
|
|
|
|||
|
|
@ -39,6 +39,17 @@ object ProtobufSerializer {
|
|||
*/
|
||||
class ProtobufSerializer(val system: ExtendedActorSystem) extends BaseSerializer {
|
||||
|
||||
@deprecated("Use constructor with ExtendedActorSystem", "2.4")
|
||||
def this() = this(null)
|
||||
|
||||
// TODO remove this when deprecated this() is removed
|
||||
override val identifier: Int =
|
||||
if (system eq null) 2
|
||||
else identifierFromConfig
|
||||
|
||||
@deprecated("Will be removed without replacement", "2.4")
|
||||
val ARRAY_OF_BYTE_ARRAY = Array[Class[_]](classOf[Array[Byte]])
|
||||
|
||||
private val parsingMethodBindingRef = new AtomicReference[Map[Class[_], Method]](Map.empty)
|
||||
|
||||
override def includeManifest: Boolean = true
|
||||
|
|
@ -52,8 +63,9 @@ class ProtobufSerializer(val system: ExtendedActorSystem) extends BaseSerializer
|
|||
parsingMethodBinding.get(clazz) match {
|
||||
case Some(cachedParsingMethod) ⇒ cachedParsingMethod
|
||||
case None ⇒
|
||||
import ProtobufSerializer.ARRAY_OF_BYTE_ARRAY
|
||||
val unCachedParsingMethod = if (method eq null) clazz.getDeclaredMethod("parseFrom", ARRAY_OF_BYTE_ARRAY: _*) else method
|
||||
val unCachedParsingMethod =
|
||||
if (method eq null) clazz.getDeclaredMethod("parseFrom", ProtobufSerializer.ARRAY_OF_BYTE_ARRAY: _*)
|
||||
else method
|
||||
if (parsingMethodBindingRef.compareAndSet(parsingMethodBinding, parsingMethodBinding.updated(clazz, unCachedParsingMethod)))
|
||||
unCachedParsingMethod
|
||||
else
|
||||
|
|
|
|||
|
|
@ -90,7 +90,18 @@ abstract class AbstractTransportAdapter(protected val wrappedTransport: Transpor
|
|||
} 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] = {
|
||||
// Prepare a future, and pass its promise to the manager
|
||||
|
|
|
|||
|
|
@ -111,7 +111,8 @@ class TestTransport(
|
|||
(_) ⇒ registry.logActivity(ShutdownAttempt(localAddress)))
|
||||
|
||||
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 shutdown(): Future[Boolean] = shutdownBehavior(())
|
||||
|
||||
|
|
|
|||
|
|
@ -104,11 +104,8 @@ trait Transport {
|
|||
*/
|
||||
def listen: Future[(Address, Promise[AssociationEventListener])]
|
||||
|
||||
/**
|
||||
* @return
|
||||
* The address this Transport is listening to.
|
||||
*/
|
||||
def boundAddress: Address
|
||||
// Need to do like this for binary compatibility reasons
|
||||
// def boundAddress: Address
|
||||
|
||||
/**
|
||||
* Asynchronously opens a logical duplex link between two Transport Entities over a network. It could be backed by a
|
||||
|
|
|
|||
|
|
@ -243,6 +243,11 @@ private[transport] object NettyTransport {
|
|||
hostName.getOrElse(sa.getAddress.getHostAddress), port.getOrElse(sa.getPort))) // perhaps use getHostString in jdk 1.7
|
||||
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
|
||||
|
|
@ -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] = {
|
||||
if (!serverChannel.isBound) Future.failed(new NettyTransportException("Transport is not bound"))
|
||||
|
|
|
|||
|
|
@ -18,6 +18,7 @@ import java.net.InetAddress;
|
|||
import java.net.InetSocketAddress;
|
||||
import java.net.NetworkInterface;
|
||||
import java.net.StandardProtocolFamily;
|
||||
import java.net.DatagramSocket;
|
||||
import java.nio.channels.DatagramChannel;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
|
@ -34,7 +35,7 @@ public class JavaUdpMulticast {
|
|||
//#inet6-protocol-family
|
||||
|
||||
//#multicast-group
|
||||
public static class MulticastGroup extends Inet.AbstractSocketOption {
|
||||
public static class MulticastGroup extends Inet.AbstractSocketOptionV2 {
|
||||
private String address;
|
||||
private String interf;
|
||||
|
||||
|
|
@ -44,11 +45,11 @@ public class JavaUdpMulticast {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void afterConnect(DatagramChannel c) {
|
||||
public void afterBind(DatagramSocket s) {
|
||||
try {
|
||||
InetAddress group = InetAddress.getByName(address);
|
||||
NetworkInterface networkInterface = NetworkInterface.getByName(interf);
|
||||
c.join(group, networkInterface);
|
||||
s.getChannel().join(group, networkInterface);
|
||||
} catch (Exception ex) {
|
||||
System.out.println("Unable to join multicast group.");
|
||||
}
|
||||
|
|
|
|||
|
|
@ -5,10 +5,11 @@
|
|||
package docs.io
|
||||
|
||||
import java.net.{InetAddress, InetSocketAddress, NetworkInterface, StandardProtocolFamily}
|
||||
import java.net.DatagramSocket
|
||||
import java.nio.channels.DatagramChannel
|
||||
|
||||
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.util.ByteString
|
||||
|
||||
|
|
@ -20,11 +21,11 @@ final case class Inet6ProtocolFamily() extends DatagramChannelCreator {
|
|||
//#inet6-protocol-family
|
||||
|
||||
//#multicast-group
|
||||
final case class MulticastGroup(address: String, interface: String) extends SocketOption {
|
||||
override def afterConnect(c: DatagramChannel) {
|
||||
final case class MulticastGroup(address: String, interface: String) extends SocketOptionV2 {
|
||||
override def afterBind(s: DatagramSocket) {
|
||||
val group = InetAddress.getByName(address)
|
||||
val networkInterface = NetworkInterface.getByName(interface)
|
||||
c.join(group, networkInterface)
|
||||
s.getChannel.join(group, networkInterface)
|
||||
}
|
||||
}
|
||||
//#multicast-group
|
||||
|
|
|
|||
|
|
@ -62,7 +62,7 @@ class TestFSMRef[S, D, T <: Actor](
|
|||
* and stop handling.
|
||||
*/
|
||||
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))
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -27,7 +27,7 @@ object AkkaBuild extends Build {
|
|||
|
||||
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
|
||||
|
||||
|
|
@ -347,13 +347,7 @@ object AkkaBuild extends Build {
|
|||
|
||||
def akkaPreviousArtifact(id: String): Def.Initialize[Option[sbt.ModuleID]] = Def.setting {
|
||||
if (enableMiMa) {
|
||||
// Note: This is a little gross because we don't have a 2.3.0 release on Scala 2.11.x
|
||||
// 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 version: String = "2.3.10" // FIXME
|
||||
val fullId = crossVersion.value match {
|
||||
case _ : CrossVersion.Binary => id + "_" + scalaBinaryVersion.value
|
||||
case _ : CrossVersion.Full => id + "_" + scalaVersion.value
|
||||
|
|
|
|||
|
|
@ -17,10 +17,506 @@ object MiMa extends AutoPlugin {
|
|||
previousArtifact := None,
|
||||
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 = {
|
||||
import com.typesafe.tools.mima.core._
|
||||
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")
|
||||
|
||||
)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue