Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
34 changes: 1 addition & 33 deletions Docs/behaviors.adoc
Original file line number Diff line number Diff line change
Expand Up @@ -38,39 +38,7 @@ This is because conceptually and Actor is the sum of three parts:
2. A api:Behavior[struct] that should be run for each subsequent message.
3. A api:MessageDispatcher[protocol] which "runs the Actor".

==== Behaviors defined in `class`-oriented style

To many developers, the more familiar way of defining api:Behavior[struct] is likely going to be the `class`-oriented style.
While we consider this state to lead into sharing too much state between the various states an actor can reside in,
we do acknowledge that the familiarity of this way of defining actors is a nice entry path to working with actors.

To define a simple behavior that will extend a `class` you can:

[source]
----
include::{dir_sact_doc_tests}/ActorDocExamples.swift[tag=classOriented_behavior]
----
<1> Extend the `ClassBehavior` base class #TODO: Maybe we call it ClassBehavior, so it's the class-oriented one? Want to avoid this being the only place with Actor in name of behavior#
<2> Implement the `receive` function; it will be invoked using the actors' context and arriving message for every message sent to this actor
<3> Do something with the message, e.g. print it
<4> Return the "next behavior" which this actor should _become_. Our actor is simple and stable in its behavior, thus it can return the special `Behavior.same`.

We can extend this actor to have some internal state, that it will keep mutating upon receipt of a message:

[source]
----
include::{dir_sact_doc_tests}/ActorDocExamples.swift[tag=classOriented_behaviorWithState]
----
<1> A class-oriented actor behavior can have stored properties which it is safe to modify during a _reduction_
<2> We increment the local counter for every message that we receive; Even though we may run these on various threads, proper synchronization is in place.

Messages will be processed one-by-one by the actor, and the _thread_ which the actor uses to run these _reductions_ MAY change in between reductions.
However, the thread-safety of accessing the actors stored property is guaranteed thanks to how actors are scheduled.

Effectively, within an actor you can work as-if it was a single threaded application. The same applies for the function-oriented style as well,
which we'll explore in the following section.

==== Behaviors defined in function-oriented style
==== Behaviors

Those three elements together form one functioning actor. In other words, an actor is a behavior that is able to be run
with messages from a mailbox. It is by design that the actor itself cannot be seen nor reached from user code, this
Expand Down
49 changes: 0 additions & 49 deletions Sources/DistributedActors/Behaviors.swift
Original file line number Diff line number Diff line change
Expand Up @@ -330,8 +330,6 @@ internal enum _Behavior<Message: ActorMessage> {
case receive(_ handle: (ActorContext<Message>, Message) throws -> Behavior<Message>)
case receiveMessage(_ handle: (Message) throws -> Behavior<Message>)

case `class`(ClassBehavior<Message>)

indirect case stop(postStop: Behavior<Message>?, reason: StopReason)
indirect case failed(behavior: Behavior<Message>, cause: Supervision.Failure)

Expand Down Expand Up @@ -383,48 +381,6 @@ public enum IllegalBehaviorError<Message: ActorMessage>: Error {
case illegalTransition(from: Behavior<Message>, to: Behavior<Message>)
}

// ==== ----------------------------------------------------------------------------------------------------------------
// MARK: Class-based Behavior

extension Behavior {
/// Allows defining actors by extending the `ClassBehavior` class.
///
/// This allows for easier storage of mutable state, since one can utilize instance variables for this,
/// rather than closing over state like it is typical in the more function heavy (class-less) style.
public static func `class`(_ makeBehavior: @escaping () -> ClassBehavior<Message>) -> Behavior<Message> {
Behavior(underlying: .setup { _ in
Behavior(underlying: .class(makeBehavior()))
})
}
}

/// Allows writing actors in "class style" by extending this behavior and spawning it using `.custom(MyBehavior())`
///
/// - SeeAlso: `Behavior` for general documentation about behaviors,
/// - SeeAlso: `Behavior.receive` and `Behavior.receiveSignal` for closure-style behaviors corresponding to the
/// `receive` and `receiveSignal` functions of the `ClassBehavior`.
open class ClassBehavior<Message: ActorMessage> {
public init() {}

/// Invoked each time the actor running this behavior is to receive a message.
/// The returned behavior will be used for handling the next incoming message or signal.
/// In order to remain the same behavior as currently, return `.same`.
///
/// - SeeAlso: `Behavior.receive`
open func receive(context: ActorContext<Message>, message: Message) throws -> Behavior<Message> {
undefined(hint: "MUST override receive(context:message:) when extending ClassBehavior")
}

/// Invoked each time the actor running this behavior is to receive a `Signal`.
/// The returned behavior will be used for handling the next incoming message or signal,
/// with the exception of signals like `Signals.PostStop`.
///
/// - SeeAlso: `Behavior.receiveSignal`
open func receiveSignal(context: ActorContext<Message>, signal: Signal) -> Behavior<Message> {
.unhandled
}
}

// ==== ----------------------------------------------------------------------------------------------------------------
// MARK: Intercepting Messages

Expand Down Expand Up @@ -508,7 +464,6 @@ public extension Behavior {
switch self.underlying {
case .receiveMessage(let recv): return try recv(message)
case .receive(let recv): return try recv(context, message)
case .class(let behavior): return try behavior.receive(context: context, message: message)
case .signalHandling(let recvMsg, _): return try recvMsg.interpretMessage(context: context, message: message) // TODO: should we keep the signal handler even if not .same? // TODO: more signal handling tests
case .intercept(let inner, let interceptor): return try Interceptor.handleMessage(context: context, behavior: inner, interceptor: interceptor, message: message)
case .orElse(let first, let second): return try self.interpretOrElse(context: context, first: first, orElse: second, message: message, file: file, line: line)
Expand Down Expand Up @@ -572,9 +527,6 @@ public extension Behavior {
case .setup:
return .unhandled

case .class(let behavior):
return behavior.receiveSignal(context: context, signal: signal)

case .same:
return .unhandled
case .ignore:
Expand Down Expand Up @@ -746,7 +698,6 @@ internal extension Behavior {
case .same: return base
case .ignore: return base
case .unhandled: return base
case .class: return canonical

case .stop(.none, let reason): return .stop(postStop: base, reason: reason)
case .stop(.some, _): return canonical
Expand Down
4 changes: 2 additions & 2 deletions Sources/DistributedActorsXPC/Serialization+XPC.swift
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ public enum XPCSerialization {
xpc_dictionary_set_uint64(xdict, ActorableXPCMessageField.serializerId.rawValue, UInt64(serialized.manifest.serializerID.value))

switch serialized.buffer {
case .data(let data):
case .data:
// FIXME: https://github.com/apple/swift-distributed-actors/issues/536
fatalError("not implmented")
case .nioByteBuffer(let buffer):
Expand All @@ -64,7 +64,7 @@ public enum XPCSerialization {
let serialized = try system.serialization.serialize(address)

switch serialized.buffer {
case .data(let data):
case .data:
// FIXME: https://github.com/apple/swift-distributed-actors/issues/536
fatalError("not implmented")
case .nioByteBuffer(let buffer):
Expand Down
25 changes: 0 additions & 25 deletions Tests/DistributedActorsDocumentationTests/ActorDocExamples.swift
Original file line number Diff line number Diff line change
Expand Up @@ -51,31 +51,6 @@ class ActorDocExamples: XCTestCase {
_ = behavior // silence not-used warning
}

func example_classOriented_behavior() throws {
// tag::classOriented_behavior[]
final class GreetingsPrinterBehavior: ClassBehavior<Greetings> { // <1>
override func receive(context: ActorContext<Greetings>, message: Greetings) throws -> Behavior<Greetings> { // <2>
print("Received: \(message)") // <3>
return .same // <4>
}
}
// end::classOriented_behavior[]
}

func example_classOriented_behaviorWithState() throws {
// tag::classOriented_behaviorWithState[]
final class GreetingsPrinterBehavior: ClassBehavior<Greetings> {
private var messageCounter = 0 // <1>

override func receive(context: ActorContext<Greetings>, message: Greetings) throws -> Behavior<Greetings> {
self.messageCounter += 1 // <2>
print("Received \(self.messageCounter)-th message: \(message)")
return .same
}
}
// end::classOriented_behaviorWithState[]
}

func example_spawn_tell() throws {
// tag::spawn[]
let system = ActorSystem("ExampleSystem") // <1>
Expand Down
43 changes: 0 additions & 43 deletions Tests/DistributedActorsTests/ActorLifecycleTests.swift
Original file line number Diff line number Diff line change
Expand Up @@ -112,47 +112,4 @@ class ActorLifecycleTests: ActorSystemTestBase {

try p.expectNoMessage(for: .milliseconds(200))
}

// ==== ------------------------------------------------------------------------------------------------------------
// MARK: Stopping actors

func test_stopping_shouldDeinitTheBehavior() throws {
let p: ActorTestProbe<String> = self.testKit.spawnTestProbe("p1")
let chattyAboutLifecycle =
try system.spawn("deinitLifecycleActor", .class { LifecycleDeinitClassBehavior(p.ref) })

chattyAboutLifecycle.tell(.stop)

try p.expectMessage("init")
try p.expectMessage("receive:stop")
try p.expectMessage("signal:PostStop()")
try p.expectMessage("deinit")
}
}

private enum LifecycleDeinitActorMessage: String, ActorMessage {
case stop
}

private final class LifecycleDeinitClassBehavior: ClassBehavior<LifecycleDeinitActorMessage> {
let probe: ActorRef<String>

init(_ p: ActorRef<String>) {
self.probe = p
self.probe.tell("init")
}

deinit {
self.probe.tell("deinit")
}

override func receive(context: ActorContext<LifecycleDeinitActorMessage>, message: LifecycleDeinitActorMessage) -> Behavior<LifecycleDeinitActorMessage> {
self.probe.tell("receive:\(message)")
return .stop
}

override func receiveSignal(context: ActorContext<LifecycleDeinitActorMessage>, signal: Signal) -> Behavior<LifecycleDeinitActorMessage> {
self.probe.tell("signal:\(signal)")
return .same
}
}
4 changes: 2 additions & 2 deletions Tests/DistributedActorsTests/BehaviorMatchers.swift
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ internal extension Behavior {
return 1 + max(try nestingDepth0(first), try nestingDepth0(other))
case .suspended(let previousBehavior, _):
return try 1 + nestingDepth0(previousBehavior)
case .same, .receive, .receiveMessage, .stop, .failed, .unhandled, .ignore, .class, .suspend:
case .same, .receive, .receiveMessage, .stop, .failed, .unhandled, .ignore, .suspend:
return 1
}
}
Expand Down Expand Up @@ -78,7 +78,7 @@ internal extension Behavior {
return "\(pad)suspended(\n" +
(try prettyFormat0(previousBehavior, depth: depth + 1)) +
"\(pad))\n"
case .same, .receive, .receiveMessage, .stop, .failed, .unhandled, .ignore, .class, .suspend:
case .same, .receive, .receiveMessage, .stop, .failed, .unhandled, .ignore, .suspend:
return "\(pad)\(b)\n"
}
}
Expand Down
95 changes: 0 additions & 95 deletions Tests/DistributedActorsTests/BehaviorTests.swift
Original file line number Diff line number Diff line change
Expand Up @@ -109,101 +109,6 @@ final class BehaviorTests: ActorSystemTestBase {
}
}

// TODO: another test with 2 senders, that either of their ordering is valid at recipient

class MyActorBehavior: ClassBehavior<TestMessage> {
public override func receive(context: ActorContext<TestMessage>, message: TestMessage) -> Behavior<TestMessage> {
message.replyTo.tell(self.thxFor(message.message))
return .same
}

func thxFor(_ m: String) -> String {
"Thanks for: <\(m)>"
}
}

// has to be ClassBehavior in test name, otherwise our generate_linux_tests is confused (and thinks this is an inner class)
func test_ClassBehavior_receivesMessages() throws {
let p: ActorTestProbe<String> = self.testKit.spawnTestProbe("testActor-5")

let ref: ActorRef<TestMessage> = try system.spawn(.anonymous, .class { MyActorBehavior() })

// first we send many messages
for i in 0 ... 10 {
ref.tell(TestMessage(message: "message-\(i)", replyTo: p.ref))
}

func thxFor(_ m: String) -> String {
"Thanks for: <\(m)>"
}

// separately see if we got the expected replies in the right order.
// we do so separately to avoid sending in "lock-step" in the first loop above here
for i in 0 ... 10 {
try p.expectMessage(thxFor("message-\(i)"))
}
}

class MySignalActorBehavior: ClassBehavior<String> {
let probe: ActorRef<Signals.Terminated>

init(probe: ActorRef<Signals.Terminated>) {
self.probe = probe
}

public override func receive(context: ActorContext<String>, message: String) throws -> Behavior<String> {
_ = try context.spawnWatch(.anonymous, Behavior<String>.stop)
return .same
}

override func receiveSignal(context: ActorContext<String>, signal: Signal) -> Behavior<String> {
if let terminated = signal as? Signals.Terminated {
self.probe.tell(terminated)
}
return .same
}
}

// has to be ClassBehavior in test name, otherwise our generate_linux_tests is confused (and thinks this is an inner class)
func test_ClassBehavior_receivesSignals() throws {
let p: ActorTestProbe<Signals.Terminated> = self.testKit.spawnTestProbe("probe-6a")
let ref: ActorRef<String> = try system.spawn(.anonymous, .class { MySignalActorBehavior(probe: p.ref) })
ref.tell("do it")

_ = try p.expectMessage()
// receiveSignal was invoked successfully
}

class MyStartingBehavior: ClassBehavior<String> {
let probe: ActorRef<String>

init(probe: ActorRef<String>) {
self.probe = probe
super.init()
self.probe.tell("init")
}

public override func receive(context: ActorContext<String>, message: String) throws -> Behavior<String> {
self.probe.tell("\(message)")
throw TestError("Boom on purpose!")
}
}

func test_ClassBehavior_executesInitOnStartSignal() throws {
let p: ActorTestProbe<String> = self.testKit.spawnTestProbe("probe-7a")
let ref: ActorRef<String> = try system.spawn(
.anonymous,
props: .supervision(strategy: .restart(atMost: 1, within: nil)),
.class { MyStartingBehavior(probe: p.ref) }
)
ref.tell("hello")

try p.expectMessage("init")
try p.expectMessage("hello")
// restarts and executes init in new instance
try p.expectMessage("init")
}

func test_receiveSpecificSignal_shouldReceiveAsExpected() throws {
let p: ActorTestProbe<Signals.Terminated> = self.testKit.spawnTestProbe("probe-specificSignal-1")
let _: ActorRef<String> = try system.spawn(
Expand Down
38 changes: 0 additions & 38 deletions Tests/DistributedActorsTests/SupervisionTests.swift
Original file line number Diff line number Diff line change
Expand Up @@ -496,28 +496,6 @@ final class SupervisionTests: ActorSystemTestBase {
)
}

func test_restartSupervised_throws_shouldRestart_andCreateNewInstanceOfClassBehavior() throws {
let p = self.testKit.spawnTestProbe(expecting: String.self)
let ref = try system.spawn(
"class-behavior",
props: .supervision(strategy: .restart(atMost: 2, within: nil)),
.class { MyCrashingClassBehavior(p.ref) }
)

ref.tell("one")
// throws and restarts
ref.tell("two")

try p.expectMessage("init")
let id1 = try p.expectMessage()
try p.expectMessage("message:one")
try p.expectMessage("init")
let id2 = try p.expectMessage()
try p.expectMessage("message:two")

id2.shouldNotEqual(id1)
}

func test_restartSupervised_throwsInAwaitResult_shouldRestart() throws {
try self.sharedTestLogic_restartSupervised_shouldRestart(
runName: "throws",
Expand All @@ -527,22 +505,6 @@ final class SupervisionTests: ActorSystemTestBase {
)
}

class MyCrashingClassBehavior: ClassBehavior<String> {
let probe: ActorRef<String>

init(_ probe: ActorRef<String>) {
self.probe = probe
super.init()
probe.tell("init")
probe.tell("\(ObjectIdentifier(self))")
}

override func receive(context: ActorContext<String>, message: String) throws -> Behavior<String> {
self.probe.tell("message:\(message)")
throw FaultyError.boom(message: "Booming on purpose, in class behavior!")
}
}

// ==== ----------------------------------------------------------------------------------------------------------------
// MARK: Escalating supervision

Expand Down