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
2 changes: 1 addition & 1 deletion Sources/ActorSingletonPlugin/ActorSingleton.swift
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ import DistributedActorsConcurrencyHelpers
// ==== ----------------------------------------------------------------------------------------------------------------
// MARK: Actor singleton

internal final class ActorSingleton<Message: ActorMessage> {
internal final class ActorSingleton<Message: Codable> {
/// Settings for the `ActorSingleton`
let settings: ActorSingletonSettings

Expand Down
4 changes: 2 additions & 2 deletions Sources/ActorSingletonPlugin/ActorSingletonManager.swift
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ import Logging

/// Spawned as a system actor on the node where the singleton is supposed to run, `ActorSingletonManager` manages
/// the singleton's lifecycle and stops itself after handing over the singleton.
internal class ActorSingletonManager<Message: ActorMessage> {
internal class ActorSingletonManager<Message: Codable> {
/// Settings for the `ActorSingleton`
private let settings: ActorSingletonSettings

Expand Down Expand Up @@ -71,7 +71,7 @@ internal class ActorSingletonManager<Message: ActorMessage> {
try context.stop(child: singleton)
}

internal enum Directive: NonTransportableActorMessage {
internal enum Directive: _NotActuallyCodableMessage {
case takeOver(from: UniqueNode?, replyTo: _ActorRef<_ActorRef<Message>?>)
case handOver(to: UniqueNode?)
case stop
Expand Down
2 changes: 1 addition & 1 deletion Sources/ActorSingletonPlugin/ActorSingletonPlugin.swift
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ public final class ActorSingletonPlugin {
public init() {}

// FIXME: document that may crash, it may right?
func ref<Message: ActorMessage>(of type: Message.Type, settings: ActorSingletonSettings, system: ClusterSystem, props: _Props? = nil, _ behavior: _Behavior<Message>? = nil) throws -> _ActorRef<Message> {
func ref<Message: Codable>(of type: Message.Type, settings: ActorSingletonSettings, system: ClusterSystem, props: _Props? = nil, _ behavior: _Behavior<Message>? = nil) throws -> _ActorRef<Message> {
try self.singletonsLock.withLock {
if let existing = self.singletons[settings.name] {
guard let proxy = existing.unsafeUnwrapAs(Message.self).proxy else {
Expand Down
2 changes: 1 addition & 1 deletion Sources/ActorSingletonPlugin/ActorSingletonProxy.swift
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ import Logging
/// singleton runs on. If the singleton falls on *this* node, the proxy will spawn a `ActorSingletonManager`,
/// which manages the actual singleton actor, and obtain the ref from it. The proxy instructs the
/// `ActorSingletonManager` to hand over the singleton whenever the node changes.
internal class ActorSingletonProxy<Message: ActorMessage> {
internal class ActorSingletonProxy<Message: Codable> {
/// Settings for the `ActorSingleton`
private let settings: ActorSingletonSettings

Expand Down
3 changes: 0 additions & 3 deletions Sources/DistributedActors/ActorAddress.swift
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,6 @@ import Distributed
/// Convenience alias for ``ClusterSystem/ActorID``.
public typealias ActorID = ClusterSystem.ActorID

@available(*, deprecated, message: "Old name of this type, to be replaced by ActorID")
public typealias ActorAddress = ActorID

extension ClusterSystem {
/// Uniquely identifies a DistributedActor within the cluster.
///
Expand Down
14 changes: 7 additions & 7 deletions Sources/DistributedActors/ActorContext.swift
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ import Logging
/// - Warning:
/// - It MUST only ever be accessed from its own Actor. It is fine though to close over it in the actors behaviours.
/// - It MUST NOT be shared to other actors, and MUST NOT be accessed concurrently (e.g. from outside the actor).
public class _ActorContext<Message: ActorMessage> /* TODO(sendable): NOTSendable*/ {
public class _ActorContext<Message: Codable> /* TODO(sendable): NOTSendable*/ {
public typealias Myself = _ActorRef<Message>

/// Returns `ClusterSystem` which this context belongs to.
Expand Down Expand Up @@ -126,7 +126,7 @@ public class _ActorContext<Message: ActorMessage> /* TODO(sendable): NOTSendable
file: String = #file, line: UInt = #line,
_ behavior: _Behavior<M>
) throws -> _ActorRef<M>
where M: ActorMessage
where M: Codable
{
_undefined()
}
Expand All @@ -145,7 +145,7 @@ public class _ActorContext<Message: ActorMessage> /* TODO(sendable): NOTSendable
file: String = #file, line: UInt = #line,
_ behavior: _Behavior<M>
) throws -> _ActorRef<M>
where M: ActorMessage
where M: Codable
{
_undefined()
}
Expand Down Expand Up @@ -174,7 +174,7 @@ public class _ActorContext<Message: ActorMessage> /* TODO(sendable): NOTSendable
/// - Throws: an `_ActorContextError` when an actor ref is passed in that is NOT a child of the current actor.
/// An actor may not terminate another's child actors. Attempting to stop `myself` using this method will
/// also throw, as the proper way of stopping oneself is returning a `_Behavior.stop`.
public func stop<M>(child ref: _ActorRef<M>) throws where M: ActorMessage {
public func stop<M>(child ref: _ActorRef<M>) throws where M: Codable {
return _undefined()
}

Expand Down Expand Up @@ -332,7 +332,7 @@ public class _ActorContext<Message: ActorMessage> /* TODO(sendable): NOTSendable
/// being silently dropped. This can be useful when not all messages `From` have a valid representation in
/// `Message`, or if not all `From` messages are of interest for this particular actor.
public final func messageAdapter<From>(_ adapt: @escaping (From) -> Message?) -> _ActorRef<From>
where From: ActorMessage
where From: Codable
{
return self.messageAdapter(from: From.self, adapt: adapt)
}
Expand All @@ -349,7 +349,7 @@ public class _ActorContext<Message: ActorMessage> /* TODO(sendable): NOTSendable
/// being silently dropped. This can be useful when not all messages `From` have a valid representation in
/// `Message`, or if not all `From` messages are of interest for this particular actor.
public func messageAdapter<From>(from type: From.Type, adapt: @escaping (From) -> Message?) -> _ActorRef<From>
where From: ActorMessage
where From: Codable
{
return _undefined()
}
Expand All @@ -365,7 +365,7 @@ public class _ActorContext<Message: ActorMessage> /* TODO(sendable): NOTSendable
/// with an existing `_SubReceiveId`, it replaces the old one. All references will remain valid and point to
/// the new behavior.
public func subReceive<SubMessage>(_: _SubReceiveId<SubMessage>, _: SubMessage.Type, _: @escaping (SubMessage) throws -> Void) -> _ActorRef<SubMessage>
where SubMessage: ActorMessage
where SubMessage: Codable
{
return _undefined()
}
Expand Down
4 changes: 1 addition & 3 deletions Sources/DistributedActors/ActorMessage+Protobuf.swift
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import SwiftProtobuf
// ==== ----------------------------------------------------------------------------------------------------------------
// MARK: Protobuf representations

public protocol _AnyProtobufRepresentable: ActorMessage, SerializationRepresentable {}
public protocol _AnyProtobufRepresentable: Codable, SerializationRepresentable {}

extension _AnyProtobufRepresentable {
public static var defaultSerializerID: Serialization.SerializerID? {
Expand All @@ -32,8 +32,6 @@ extension _AnyProtobufRepresentable {
public protocol _AnyPublicProtobufRepresentable: _AnyProtobufRepresentable {}

/// A protocol that facilitates conversion between Swift and protobuf messages.
///
/// - SeeAlso: `ActorMessage`
public protocol _ProtobufRepresentable: _AnyPublicProtobufRepresentable {
associatedtype ProtobufRepresentation: SwiftProtobuf.Message

Expand Down
34 changes: 11 additions & 23 deletions Sources/DistributedActors/ActorMessages.swift
Original file line number Diff line number Diff line change
Expand Up @@ -15,25 +15,14 @@
import struct Foundation.Data
import NIO

// ==== ----------------------------------------------------------------------------------------------------------------
// MARK: Actor Message

/// An Actor message is simply a Codable type.
///
/// Any Codable it able to be sent as an actor message.
///
/// You can customize which coder/decoder should be used by registering specialized manifests for the message type,
/// or having the type conform to one of the special `...Representable` (e.g. `_ProtobufRepresentable`) protocols.
public typealias ActorMessage = Codable // FIXME: MAKE THIS SENDABLE: & Sendable

/// A `Never` can never be sent as message, even more so over the wire.
extension Never: NonTransportableActorMessage {}
extension Never: _NotActuallyCodableMessage {}

// ==== ----------------------------------------------------------------------------------------------------------------
// MARK: Common utility messages

// FIXME: we should not add Codable conformance onto a stdlib type, but rather fix this in stdlib
extension Result: ActorMessage where Success: ActorMessage, Failure: ActorMessage {
extension Result: Codable where Success: Codable, Failure: Codable {
public enum DiscriminatorKeys: String, Codable {
case success
case failure
Expand Down Expand Up @@ -69,7 +58,7 @@ extension Result: ActorMessage where Success: ActorMessage, Failure: ActorMessag
}

/// Generic transportable Error type, can be used to wrap error types and represent them as best as possible for transporting.
public struct ErrorEnvelope: Error, ActorMessage {
public struct ErrorEnvelope: Error, Codable {
public typealias CodableError = Error & Codable

private let codableError: CodableError
Expand Down Expand Up @@ -139,7 +128,7 @@ public struct BestEffortStringError: Error, Codable, Equatable, CustomStringConv
}

/// Useful error wrapper which performs an best effort Error serialization as configured by the actor system.
public struct NonTransportableAnyError: Error, NonTransportableActorMessage {
public struct NonTransportableAnyError: Error, _NotActuallyCodableMessage {
public let failure: Error

public init<Failure: Error>(_ failure: Failure) {
Expand All @@ -150,8 +139,7 @@ public struct NonTransportableAnyError: Error, NonTransportableActorMessage {
// ==== ----------------------------------------------------------------------------------------------------------------
// MARK: Not Transportable Actor Message (i.e. "local only")

/// Marks a type as `ActorMessage` however
/// Attempting to send such message to a remote actor WILL FAIL and log an error.
/// Marks a type as `Codable` however attempting to send such message to a remote actor WILL FAIL and log an error.
///
/// Use this with great caution and only for messages which are specifically designed to utilize the local assumption.
///
Expand All @@ -162,22 +150,22 @@ public struct NonTransportableAnyError: Error, NonTransportableActorMessage {
/// No serializer is expected to be registered for such types.
///
/// - Warning: Attempting to send such message over the network will fail at runtime (and log an error or warning).
public protocol NonTransportableActorMessage: ActorMessage {}
public protocol _NotActuallyCodableMessage: Codable {}

extension NonTransportableActorMessage {
extension _NotActuallyCodableMessage {
public init(from decoder: Swift.Decoder) throws {
fatalError("Attempted to decode NonTransportableActorMessage message: \(Self.self)! This should never happen.")
fatalError("Attempted to decode _NotActuallyCodableMessage message: \(Self.self)! This should never happen.")
}

public func encode(to encoder: Swift.Encoder) throws {
fatalError("Attempted to encode NonTransportableActorMessage message: \(Self.self)! This should never happen.")
fatalError("Attempted to encode _NotActuallyCodableMessage message: \(Self.self)! This should never happen.")
}

public init(context: Serialization.Context, from buffer: inout ByteBuffer, using manifest: Serialization.Manifest) throws {
fatalError("Attempted to deserialize NonTransportableActorMessage message: \(Self.self)! This should never happen.")
fatalError("Attempted to deserialize _NotActuallyCodableMessage message: \(Self.self)! This should never happen.")
}

public func serialize(context: Serialization.Context, to bytes: inout ByteBuffer) throws {
fatalError("Attempted to serialize NonTransportableActorMessage message: \(Self.self)! This should never happen.")
fatalError("Attempted to serialize _NotActuallyCodableMessage message: \(Self.self)! This should never happen.")
}
}
19 changes: 2 additions & 17 deletions Sources/DistributedActors/ActorRef+Ask.swift
Original file line number Diff line number Diff line change
Expand Up @@ -86,12 +86,8 @@ extension _ActorRef: ReceivesQuestions {

let promise = system._eventLoopGroup.next().makePromise(of: answerType)

// TODO: maybe a specialized one... for ask?
let instrumentation = system.settings.instrumentation.makeActorInstrumentation(promise.futureResult, self.id)

do {
// TODO: implement special actor ref instead of using real actor
let askRef = try system._spawn(
let askRef = try system._spawn( // TODO: "ask" is going away in favor of raw "remoteCalls"
.ask,
AskActor.behavior(
promise,
Expand All @@ -105,18 +101,7 @@ extension _ActorRef: ReceivesQuestions {

let message = makeQuestion(askRef)
self.tell(message, file: file, line: line)

instrumentation.actorAsked(message: message, from: askRef.id)
promise.futureResult.whenComplete {
switch $0 {
case .success(let answer):
instrumentation.actorAskReplied(reply: answer, error: nil)
case .failure(let error):
instrumentation.actorAskReplied(reply: nil, error: error)
}
}
} catch {
instrumentation.actorAskReplied(reply: nil, error: error)
promise.fail(error)
}

Expand Down Expand Up @@ -253,7 +238,7 @@ extension AskResponse {
///
// TODO: replace with a special minimal `_ActorRef` that does not require spawning or scheduling.
internal enum AskActor {
enum Event: NonTransportableActorMessage {
enum Event: _NotActuallyCodableMessage {
case timeout
}

Expand Down
4 changes: 2 additions & 2 deletions Sources/DistributedActors/ActorRefFactory.swift
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ public protocol _ActorRefFactory {
props: _Props,
file: String, line: UInt,
_ behavior: _Behavior<Message>
) throws -> _ActorRef<Message> where Message: ActorMessage
) throws -> _ActorRef<Message> where Message: Codable
}

// ==== ----------------------------------------------------------------------------------------------------------------
Expand All @@ -64,7 +64,7 @@ public protocol _ChildActorRefFactory: _ActorRefFactory {
var children: _Children { get set } // lock-protected

func stop<Message>(child ref: _ActorRef<Message>) throws
where Message: ActorMessage
where Message: Codable
}

// ==== ----------------------------------------------------------------------------------------------------------------
Expand Down
Loading