Skip to content

Commit

Permalink
Merge pull request akka#17354 from akka/wip-17342-bc-patriknw
Browse files Browse the repository at this point in the history
= akka#17342 Make 2.4 binary compatible with 2.3
  • Loading branch information
patriknw committed May 12, 2015
2 parents 85860ac + b30e460 commit 00a481a
Show file tree
Hide file tree
Showing 56 changed files with 1,150 additions and 197 deletions.
4 changes: 2 additions & 2 deletions akka-actor-tests/src/test/scala/akka/event/EventBusSpec.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down Expand Up @@ -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)
Expand Down
13 changes: 7 additions & 6 deletions akka-actor-tests/src/test/scala/akka/io/UdpIntegrationSpec.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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)
Expand All @@ -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
}
}
Expand Down
3 changes: 3 additions & 0 deletions akka-actor/build.sbt
Original file line number Diff line number Diff line change
Expand Up @@ -14,3 +14,6 @@ OSGi.actor
libraryDependencies ++= Dependencies.actor

MimaKeys.previousArtifact := akkaPreviousArtifact("akka-actor").value

spray.boilerplate.BoilerplatePlugin.Boilerplate.settings

Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
/**
* Copyright (C) 2015 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.japi.function

[3..22#/**
* A Function interface. Used to create 1-arg first-class-functions is Java.
* `Serializable` is needed to be able to grab line number for Java 8 lambdas.
*/
@SerialVersionUID(##1L)
trait Function1[[#-T1#], +R] extends java.io.Serializable {
@throws(classOf[Exception])
def apply([#arg1: T1#]): R
}#

]

[2..#/**
* A Consumer interface. Used to create 1-arg consumers in Java.
* A Procedure is like a Function, but it doesn't produce a return value.
* `Serializable` is needed to be able to grab line number for Java 8 lambdas.
*/
@SerialVersionUID(##1L)
trait Procedure1[[#-T1#]] extends java.io.Serializable {
@throws(classOf[Exception])
def apply([#arg1: T1#]): Unit
}#

]
3 changes: 3 additions & 0 deletions akka-actor/src/main/scala/akka/actor/ActorPath.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
40 changes: 33 additions & 7 deletions akka-actor/src/main/scala/akka/actor/FSM.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}

/**
Expand Down Expand Up @@ -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)
}
}

Expand Down Expand Up @@ -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.
Expand All @@ -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.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -381,7 +381,10 @@ object ForkJoinExecutorConfigurator {
unhandledExceptionHandler: Thread.UncaughtExceptionHandler) = this(parallelism, threadFactory, unhandledExceptionHandler, asyncMode = true)

override def execute(r: Runnable): Unit =
if (r eq null) throw new NullPointerException else super.execute(new AkkaForkJoinTask(r))
if (r ne null)
super.execute((if (r.isInstanceOf[ForkJoinTask[_]]) r else new AkkaForkJoinTask(r)).asInstanceOf[ForkJoinTask[Any]])
else
throw new NullPointerException("Runnable was null")

def atFullThrottle(): Boolean = this.getActiveThreadCount() >= this.getParallelism()
}
Expand Down
21 changes: 18 additions & 3 deletions akka-actor/src/main/scala/akka/dispatch/Mailbox.scala
Original file line number Diff line number Diff line change
Expand Up @@ -11,8 +11,8 @@ import akka.actor.{ ActorCell, ActorRef, Cell, ActorSystem, InternalActorRef, De
import akka.util.{ BoundedBlockingQueue, StablePriorityBlockingQueue, StablePriorityQueue, Unsafe }
import akka.util.Helpers.ConfigOps
import akka.event.Logging.Error
import scala.concurrent.duration.Duration
import scala.concurrent.duration.FiniteDuration
import scala.concurrent.duration.{ Duration, FiniteDuration }
import scala.concurrent.forkjoin.ForkJoinTask
import scala.annotation.tailrec
import scala.util.control.NonFatal
import com.typesafe.config.Config
Expand Down Expand Up @@ -53,7 +53,7 @@ private[akka] object Mailbox {
* INTERNAL API
*/
private[akka] abstract class Mailbox(val messageQueue: MessageQueue)
extends SystemMessageQueue with Runnable {
extends ForkJoinTask[Unit] with SystemMessageQueue with Runnable {

import Mailbox._

Expand Down Expand Up @@ -228,6 +228,21 @@ private[akka] abstract class Mailbox(val messageQueue: MessageQueue)
}
}

override final def getRawResult(): Unit = ()
override final def setRawResult(unit: Unit): Unit = ()
final override def exec(): Boolean = try { run(); false } catch {
case ie: InterruptedException
Thread.currentThread.interrupt()
false
case anything: Throwable
val t = Thread.currentThread
t.getUncaughtExceptionHandler match {
case null
case some some.uncaughtException(t, anything)
}
throw anything
}

/**
* Process the messages in the mailbox
*/
Expand Down
126 changes: 119 additions & 7 deletions akka-actor/src/main/scala/akka/event/EventBus.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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 =
Expand Down Expand Up @@ -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]

Expand Down Expand Up @@ -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)
}
Loading

0 comments on commit 00a481a

Please sign in to comment.