From d82c40eddf9ef7caff4329de23e521c0cfee5876 Mon Sep 17 00:00:00 2001 From: Konrad `ktoso` Malawski Date: Mon, 27 Apr 2020 11:41:08 +0900 Subject: [PATCH 01/18] =remote #482 #382 #383 Race free association / remote sends --- Sources/DistributedActors/ActorShell.swift | 19 -- Sources/DistributedActors/ActorSystem.swift | 3 +- .../Cluster/Association.swift | 192 +++++++---- .../Cluster/ClusterShell+LeaderActions.swift | 17 +- .../Cluster/ClusterShell.swift | 308 ++++++++++-------- .../Cluster/ClusterShellState.swift | 76 +---- .../Cluster/HandshakeStateMachine.swift | 2 +- .../Cluster/SWIM/SWIMInstance.swift | 10 +- .../Cluster/SWIM/SWIMShell.swift | 15 +- .../Cluster/SystemMessages+Redelivery.swift | 3 +- .../ActorRef+RemotePersonality.swift | 148 --------- .../RemoteClusterActorPersonality.swift | 114 +++++++ .../Transport/TransportPipelines.swift | 2 +- Sources/DistributedActors/Receptionist.swift | 2 +- Sources/DistributedActors/Refs+any.swift | 6 +- Sources/DistributedActors/Refs.swift | 8 +- .../Cluster/ClusteredNodesTestBase.swift | 2 +- .../ActorSystem+Testing.swift | 2 +- .../Cluster/AssociationClusteredTests.swift | 42 ++- .../ClusterLeaderActionsClusteredTests.swift | 10 +- .../Cluster/RemoteActorRefProviderTests.swift | 7 +- .../ShootTheOtherNodeClusteredTests.swift | 6 +- 22 files changed, 493 insertions(+), 501 deletions(-) delete mode 100644 Sources/DistributedActors/Cluster/Transport/ActorRef+RemotePersonality.swift create mode 100644 Sources/DistributedActors/Cluster/Transport/RemoteClusterActorPersonality.swift diff --git a/Sources/DistributedActors/ActorShell.swift b/Sources/DistributedActors/ActorShell.swift index af714c0ca..ae3d5ac04 100644 --- a/Sources/DistributedActors/ActorShell.swift +++ b/Sources/DistributedActors/ActorShell.swift @@ -635,16 +635,6 @@ public final class ActorShell: ActorContext, Abs return try self._spawn(naming, props: props, behavior) } -// public override func spawn( -// _ naming: ActorNaming, of type: M.Type = M.self, props: Props = Props(), -// file: String = #file, line: UInt = #line, -// _ behavior: Behavior -// ) throws -> ActorRef -// where M: ActorMessage { -// try self.system.serialization._ensureCodableSerializer(type, file: file, line: line) -// return try self._spawn(naming, props: props, behavior) -// } - public override func spawnWatch( _ naming: ActorNaming, of type: Message.Type = Message.self, props: Props, file: String = #file, line: UInt = #line, @@ -654,15 +644,6 @@ public final class ActorShell: ActorContext, Abs self.watch(try self.spawn(naming, props: props, behavior)) } -// public override func spawnWatch( -// _ naming: ActorNaming, of type: Message.Type = Message.self, props: Props, -// file: String = #file, line: UInt = #line, -// _ behavior: Behavior -// ) throws -> ActorRef -// where Message: ActorMessage { -// self.watch(try self.spawn(naming, props: props, behavior)) -// } - public override func stop(child ref: ActorRef) throws { try self._stop(child: ref) } diff --git a/Sources/DistributedActors/ActorSystem.swift b/Sources/DistributedActors/ActorSystem.swift index 308bf4113..b3bc0bdc2 100644 --- a/Sources/DistributedActors/ActorSystem.swift +++ b/Sources/DistributedActors/ActorSystem.swift @@ -221,8 +221,7 @@ public final class ActorSystem { var effectiveSystemProvider: _ActorRefProvider = localSystemProvider if settings.cluster.enabled { - // FIXME: make SerializationPoolSettings configurable - let cluster = ClusterShell() + let cluster = ClusterShell(selfNode: settings.cluster.uniqueBindNode) initializationLock.withWriterLockVoid { self._cluster = cluster } diff --git a/Sources/DistributedActors/Cluster/Association.swift b/Sources/DistributedActors/Cluster/Association.swift index 611fe4110..f4afd5c2d 100644 --- a/Sources/DistributedActors/Cluster/Association.swift +++ b/Sources/DistributedActors/Cluster/Association.swift @@ -12,6 +12,7 @@ // //===----------------------------------------------------------------------===// +import DistributedActorsConcurrencyHelpers import Logging import NIO @@ -20,112 +21,163 @@ import NIO /// An `Association` represents a bi-directional agreement between two nodes that they are able to communicate with each other. /// -/// All interactions with a remote node MUST be driven through an association, as otherwise one might write to a node -/// that would not acknowledge any of our messages. This is important for example if a remote node is terminated, -/// and another node is brought up on the exact same network `Node` -- thus the need to keep a `UniqueNode` of -/// both "sides" of an association -- we want to inform a remote node about our identity, and want to confirm if the remote -/// sending side of an association remains the "same exact node", or if it is a new instance on the same address. +/// An association MUST be obtained with a node before any message exchange with it may occur, regardless what transport the +/// association ends up using. The association upholds the "associated only once" as well as message order delivery guarantees +/// towards the target node. /// -/// An `Association` can only be obtained by successfully completing a `HandshakeStateMachine` dance. +/// All interactions with a remote node MUST be driven through an association. +/// This is important for example if a remote node is terminated, and another node is brought up on the exact same network `Node` -- +/// thus the need to keep a `UniqueNode` of both "sides" of an association -- we want to inform a remote node about our identity, +/// and want to confirm if the remote sending side of an association remains the "same exact node", or if it is a new instance on the same address. +/// +/// A completed ("associated") `Association` can ONLY be obtained by successfully completing a `HandshakeStateMachine` dance, +/// as only the handshake can ensure that the other side is also an actor node that is able and willing to communicate with us. struct Association { - enum State { - case associated(AssociatedState) - // case leaving // so we can receive that another node saw us as DOWN - case tombstone(TombstoneState) - } - - struct AssociatedState: CustomStringConvertible { - let log: Logger - - // Mutable since we may need to reconnect and swap for a different channel? - var channel: Channel + // TODO: reword all docs; association from now on means "from init until valid and reconnections as well" + final class AssociationState: CustomStringConvertible { + // TODO: Terrible lock which we want to get rid of; it means that every remote send has to content against all other sends about getting this ref + // and the only reason is really because the off chance case in which we have to make an Association earlier than we have the handshake completed (i.e. we send to a ref that is not yet associated) + let lock: Lock + + // TODO: This style of implementation queue -> channel swapping can only ever work with coarse locking and is just temporary + // We'd prefer to have a lock-less way to implement this and we can achieve it but it's a pain to implement so will be done in a separate step. + var state: State + + enum State { + case associating(queue: MPSCLinkedQueue) + case associated(channel: Channel) // TODO: ActorTransport.Node/Peer/Target ??? + case tombstone(ActorRef) + } /// The address of this node, that was offered to the remote side for this association /// This matters in case we have multiple "self" addresses; e.g. we bind to one address, but expose another because NAT let selfNode: UniqueNode var remoteNode: UniqueNode - init(fromCompleted handshake: HandshakeStateMachine.CompletedState, log: Logger, over channel: Channel) { - self.log = log - self.remoteNode = handshake.remoteNode - self.selfNode = handshake.localNode - self.channel = channel + init(selfNode: UniqueNode, remoteNode: UniqueNode) { + self.selfNode = selfNode + self.remoteNode = remoteNode + self.lock = Lock() + self.state = .associating(queue: .init()) } - func makeRemoteControl() -> AssociationRemoteControl { - AssociationRemoteControl(channel: self.channel, remoteNode: self.remoteNode) - // TODO: RemoteControl should mimic what the ClusterShell does when it sends messages; we want to push + /// Complete the association and drain any pending message sends onto the channel. + // TODO: This style can only ever work since we lock around the entirety of enqueueing messages and this setting; make it such that we don't need the lock eventually + func completeAssociation(handshake: HandshakeStateMachine.CompletedState, over channel: Channel) { + // TODO: assert that the channel is for the right remote node? + + self.lock.withLockVoid { + switch self.state { + case .associating(let sendQueue): + // 1) store associated channel + self.state = .associated(channel: channel) + + // 2) we need to flush all the queued up messages + // - yes, we need to flush while holding the lock... it's an annoyance in this lock based design + // but it ensures that once we've flushed, all other messages will be sent in the proper order "after" + // the previously enqueued ones; A lockless design would not be able to get rid of the queue AFAIR, + while let envelope = sendQueue.dequeue() { + _ = channel.writeAndFlush(envelope) + } + + case .associated(let existingAssociatedChannel): + fatalError("MUST NOT complete an association twice; Was \(existingAssociatedChannel) and tried to complete with \(channel) from \(handshake)") + + case .tombstone: + _ = channel.close() + return + } + } } - func makeTombstone(system: ActorSystem) -> TombstoneState { - // TODO: we pass the system so we can switch to system.time for calculations in the future - TombstoneState(fromAssociated: self, settings: system.settings.cluster) + /// Terminate the association and store a tombstone in it. + /// + /// If any messages were still queued up in it, or if it was hosting a channel these get drained / closed, + /// before the tombstone is returned. + /// + /// After invoking this the association will never again be useful for sending messages. + func terminate(_ system: ActorSystem) -> Association.Tombstone { + self.lock.withLockVoid { + switch self.state { + case .associating(let sendQueue): + while let envelope = sendQueue.dequeue() { + system.deadLetters.tell(.init(envelope.underlyingMessage, recipient: envelope.recipient)) + } + // in case someone stored a reference to this association in a ref, we swap it into a dead letter sink + self.state = .tombstone(system.deadLetters) + case .associated(let channel): + _ = channel.close() + // in case someone stored a reference to this association in a ref, we swap it into a dead letter sink + self.state = .tombstone(system.deadLetters) + case .tombstone: + () // ok + } + } + + return Association.Tombstone(self.remoteNode, settings: system.settings.cluster) } var description: String { - "AssociatedState(channel: \(self.channel), selfNode: \(self.selfNode), remoteNode: \(self.remoteNode))" + "AssociatedState(\(self.state), selfNode: \(self.selfNode), remoteNode: \(self.remoteNode))" + } + } +} + +extension Association.AssociationState { + /// Concurrency: safe to invoke from any thread. + func sendUserMessage(envelope: Envelope, recipient: ActorAddress, promise: EventLoopPromise? = nil) { + let transportEnvelope = TransportEnvelope(envelope: envelope, recipient: recipient) + self._send(transportEnvelope, promise: promise) + } + + /// Concurrency: safe to invoke from any thread. + func sendSystemMessage(_ message: _SystemMessage, recipient: ActorAddress, promise: EventLoopPromise? = nil) { + let transportEnvelope = TransportEnvelope(systemMessage: message, recipient: recipient) + self._send(transportEnvelope, promise: promise) + } + + /// Concurrency: safe to invoke from any thread. + // TODO: Reimplement association such that we don't need locks here + private func _send(_ envelope: TransportEnvelope, promise: EventLoopPromise?) { + self.lock.withLockVoid { + switch self.state { + case .associating(let sendQueue): + sendQueue.enqueue(envelope) + case .associated(let channel): + channel.writeAndFlush(envelope, promise: promise) + case .tombstone(let deadLetters): + deadLetters.tell(.init(envelope.underlyingMessage, recipient: envelope.recipient)) + } } } +} - struct TombstoneState: Hashable { +extension Association { + struct Tombstone: Hashable { let remoteNode: UniqueNode /// Determines when the Tombstone should be removed from kept tombstones in the ClusterShell. /// End of life of the tombstone is calculated as `now + settings.associationTombstoneTTL`. - let removalDeadline: Deadline + let removalDeadline: Deadline // TODO: cluster should have timer to try to remove those periodically - init(fromAssociated associated: AssociatedState, settings: ClusterSettings) { + init(_ node: UniqueNode, settings: ClusterSettings) { // TODO: if we made system carry system.time we could always count from that point in time with a TimeAmount; require Clock and settings then self.removalDeadline = Deadline.fromNow(settings.associationTombstoneTTL) - self.remoteNode = associated.remoteNode + self.remoteNode = node } /// Used to create "any" tombstone, for being able to lookup in Set - init(remoteNode: UniqueNode) { + init(_ node: UniqueNode) { self.removalDeadline = Deadline.uptimeNanoseconds(1) // ANY value here is ok, we do not use it in hash/equals - self.remoteNode = remoteNode + self.remoteNode = node } func hash(into hasher: inout Hasher) { self.remoteNode.hash(into: &hasher) } - static func == (lhs: TombstoneState, rhs: TombstoneState) -> Bool { + static func == (lhs: Tombstone, rhs: Tombstone) -> Bool { lhs.remoteNode == rhs.remoteNode } } } - -// ==== ---------------------------------------------------------------------------------------------------------------- -// MARK: Control capabilities (exposed to RemoteActorRef) - -/// A "Remote Control" offered to actors which wish to perform actions onto an association, e.g. send messages to the remote side. -/// Several remote controls may be offered to actor refs, however the underlying resource is only one (like many ways to control one Apple TV). -/// -// Implementation notes: -// This is triggered when we want to send to a remote actor, and the `RemoteActorRef` triggers the sends here. -// This has to multiplex all the writes into: serialization and writing the message to the right -// - single remote control, and many writers to it, -// - they enqueue to a local queue form which messages shall be pulled into the pipeline -internal struct AssociationRemoteControl { - private let channel: Channel - let remoteNode: UniqueNode - - init(channel: Channel, remoteNode: UniqueNode) { - self.channel = channel - self.remoteNode = remoteNode - } - - func sendUserMessage(envelope: Envelope, recipient: ActorAddress, promise: EventLoopPromise? = nil) { - let transportEnvelope = TransportEnvelope(envelope: envelope, recipient: recipient) - self.channel.writeAndFlush(NIOAny(transportEnvelope), promise: promise) - } - - func sendSystemMessage(_ message: _SystemMessage, recipient: ActorAddress, promise: EventLoopPromise? = nil) { - self.channel.writeAndFlush(NIOAny(TransportEnvelope(systemMessage: message, recipient: recipient)), promise: promise) - } - - func closeChannel() -> EventLoopFuture { - self.channel.close() - } -} diff --git a/Sources/DistributedActors/Cluster/ClusterShell+LeaderActions.swift b/Sources/DistributedActors/Cluster/ClusterShell+LeaderActions.swift index e956833d4..83e4ae36a 100644 --- a/Sources/DistributedActors/Cluster/ClusterShell+LeaderActions.swift +++ b/Sources/DistributedActors/Cluster/ClusterShell+LeaderActions.swift @@ -158,22 +158,7 @@ extension ClusterShell { state._latestGossip.incrementOwnerVersion() state.gossipControl.update(payload: state._latestGossip) - switch state.association(with: memberToRemove.node.node) { - case .some(.associated(let associated)): - self.terminateAssociation(system, state: &state, associated) - case .some(.tombstone(let tombstone)): - state.log.trace("Attempted to remove association but not associated (already tombstoned): \(memberToRemove), tombstone: \(tombstone)") - case .none: - // very carefully ensure that even though it was not associated, we DO store a tombstone for it -- this is in case - // we are racing against establishing an association with a node that we already know should be dead (and for some reason it'd reply to our handshake (zombie!)). - if let removed = state.removeAssociation(system, associatedNode: memberToRemove.node) { - self.finishTerminateAssociation(system, state: &state, removalDirective: removed) - } else { - let enforceTombstone = ClusterShellState.RemoveAssociationDirective(removedAssociation: nil, tombstone: .init(remoteNode: memberToRemove.node)) - state.log.warning("Attempted to remove association but not associated NOR tombstoned: \(memberToRemove), attempting to tombstone: \(enforceTombstone)") - self.finishTerminateAssociation(system, state: &state, removalDirective: enforceTombstone) - } - } + self.terminateAssociation(system, state: &state, memberToRemove.node) state.log.info( "Leader removed member: \(memberToRemove), all nodes are certain to have seen it as [.down] before", diff --git a/Sources/DistributedActors/Cluster/ClusterShell.swift b/Sources/DistributedActors/Cluster/ClusterShell.swift index 7e9e94e8a..0bc575777 100644 --- a/Sources/DistributedActors/Cluster/ClusterShell.swift +++ b/Sources/DistributedActors/Cluster/ClusterShell.swift @@ -32,88 +32,93 @@ internal class ClusterShell { internal static let naming = ActorNaming.unique("cluster") public typealias Ref = ActorRef + private let selfNode: UniqueNode + // ~~~~~~ HERE BE DRAGONS, shared concurrently modified concurrent state ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ // We do this to avoid "looping" any initial access of an actor ref through the cluster shell's mailbox // which would cause more latency to obtaining the association. refs cache the remote control once they have obtained it. + // TODO: consider ReadWriteLock lock, these accesses are very strongly read only biased private let _associationsLock: Lock + /// Used by remote actor refs to obtain associations /// - Protected by: `_associationsLock` - private var _associationsRegistry: [UniqueNode: AssociationRemoteControl] - /// Tombstoned nodes are kept here in order to avoid attempting to associate if we get a reference to such node, + private var _activeAssociations: [UniqueNode: Association.AssociationState] + /// Node tombstones are kept here in order to avoid attempting to associate if we get a reference to such node, /// which would normally trigger an `ensureAssociated`. /// - Protected by: `_associationsLock` - private var _associationTombstones: Set + private var _associationTombstones: [UniqueNode: Association.Tombstone] - private var _swimRef: SWIM.Ref? - - private var clusterEvents: EventStream! - - // `_serializationPool` is only used when `start()` is invoked, and there it is set immediately as well - // any earlier access to the pool is a bug (in our library) and must be treated as such. - private var _serializationPool: SerializationPool? - internal var serializationPool: SerializationPool { - guard let pool = self._serializationPool else { - fatalError("BUG! Tried to access serializationPool on \(self) and it was nil! Please report this on the issue tracker.") + internal func getExistingAssociation(with node: Node) -> Association.AssociationState? { + self._associationsLock.withLock { + // TODO: a bit terrible; perhaps we should key be Node and then confirm by UniqueNode? + // this used to be separated in the State keeping them by Node and here we kept by unique though that caused other challenges + self._activeAssociations.first { + key, _ in key.node == node + }?.value } - return pool - } - - /// MUST be called while holding `_associationsLock`. - internal func _associationHasTombstone(node: UniqueNode) -> Bool { - self._associationTombstones.contains(.init(remoteNode: node)) } + /// Get an existing association or ensure that a new one shall be stored and joining kicked off if the target node was not known yet. /// Safe to concurrently access by privileged internals directly - internal func associationRemoteControl(with node: UniqueNode) -> AssociationRemoteControlState { + internal func getEnsureAssociation(with node: UniqueNode) -> StoredAssociationState { self._associationsLock.withLock { - guard !self._associationHasTombstone(node: node) else { - return .tombstone - } - guard let association = self._associationsRegistry[node] else { - return .unknown + if let tombstone = self._associationTombstones[node] { + return .tombstone(tombstone) + } else if let existing = self._activeAssociations[node] { + return .association(existing) + } else { + let association = Association.AssociationState(selfNode: self.selfNode, remoteNode: node) + self._activeAssociations[node] = association + + /// We're trying to send to `node` yet it has no association (not even in progress), + /// thus we need to kick it off. Once it completes it will .completeAssociation() on the stored one (here in the field in Shell). + self.ref.tell(.command(.handshakeWith(node.node, replyTo: nil))) + + return .association(association) } - return .associated(association) } } - enum AssociationRemoteControlState { - case unknown - case associated(AssociationRemoteControl) - case tombstone + enum StoredAssociationState { + /// An existing (ready or being associated association) which can be used to send (or buffer buffer until associated/terminated) + case association(Association.AssociationState) + /// The association with the node existed, but is now a tombstone and no more messages shall be send to it. + case tombstone(Association.Tombstone) } - /// Terminate an association including its connection, and store a tombstone for it - internal func terminateAssociation(_ system: ActorSystem, state: inout ClusterShellState, _ associated: Association.AssociatedState) { - traceLog_Remote(system.cluster.node, "Terminate association [\(associated.remoteNode)]") + /// To be invoked by cluster shell whenever an association is made. + /// Causes messages to be flushed onto the new associated channel. + private func storeCompleteAssociation(_ associated: ClusterShellState.AssociatedDirective) { + self._associationsLock.withLockVoid { + let association = self._activeAssociations[associated.handshake.remoteNode] ?? + Association.AssociationState(selfNode: associated.handshake.localNode, remoteNode: associated.handshake.remoteNode) - if let removalDirective = state.removeAssociation(system, associatedNode: associated.remoteNode) { - self.finishTerminateAssociation(system, state: &state, removalDirective: removalDirective) - } else { - () // no association to remove, thus nothing to act on + association.completeAssociation(handshake: associated.handshake, over: associated.channel) + + self._activeAssociations[associated.handshake.remoteNode] = association } } /// Performs all cleanups related to terminating an association: /// - cleans the Shell local Association cache - /// - sets a tombstone for the now-tombstoned UniqueNode + /// - sets a tombstone for the now-tombstone UniqueNode /// - ensures node is at least .down in the Membership /// /// Can be invoked as result of a direct .down being issued, or because of a node replacement happening. - internal func finishTerminateAssociation(_ system: ActorSystem, state: inout ClusterShellState, removalDirective: ClusterShellState.RemoveAssociationDirective) { - traceLog_Remote(system.cluster.node, "Finish terminate association [\(removalDirective.tombstone.remoteNode)]") - let remoteNode = removalDirective.tombstone.remoteNode + internal func terminateAssociation(_ system: ActorSystem, state: inout ClusterShellState, _ remoteNode: UniqueNode) { + traceLog_Remote(system.cluster.node, "Terminate association with [\(remoteNode)]") - // tombstone the association in the shell immediately. - // No more message sends to the system will be possible. - self._associationsLock.withLockVoid { + let removedAssociationOption: Association.AssociationState? = self._associationsLock.withLock { + // tombstone the association in the shell immediately. + // No more message sends to the system will be possible. traceLog_Remote(system.cluster.node, "Finish terminate association [\(remoteNode)]: Stored tombstone") - self._associationsRegistry.removeValue(forKey: remoteNode) - self._associationTombstones.insert(removalDirective.tombstone) + self._associationTombstones[remoteNode] = Association.Tombstone(remoteNode, settings: system.settings.cluster) + return self._activeAssociations.removeValue(forKey: remoteNode) } - // if the association was removed, we need to close it and ensure that other layers are synced up with this decision - guard let removedAssociation = removalDirective.removedAssociation else { + guard let removedAssociation = removedAssociationOption else { + system.log.warning("Attempted to terminate non-existing association [\(remoteNode)].") return } @@ -127,16 +132,13 @@ internal class ClusterShell { // it was already removed, nothing to do state.log.trace( "Finish association with \(remoteNode), yet node not in membership already?", - metadata: [ - "cluster/membership": "\(state.membership)", - ] + metadata: ["cluster/membership": "\(state.membership)"] ) } // else: Note that we CANNOT remove() just yet, as we only want to do this when all nodes have seen the down/leaving - // The lat thing we attempt to do with the other node is to shoot it, in case it's a "zombie" that still may - // receive messages for some reason. - let remoteControl = removedAssociation.makeRemoteControl() - ClusterShell.shootTheOtherNodeAndCloseConnection(system: system, targetNodeRemoteControl: remoteControl) + // The last thing we attempt to do with the other node is to shoot it, + // in case it's a "zombie" that still may receive messages for some reason. + ClusterShell.shootTheOtherNodeAndCloseConnection(system: system, targetNodeAssociation: removedAssociation) } /// Final action performed when severing ties with another node. @@ -145,17 +147,17 @@ internal class ClusterShell { /// This is a best-effort message; as we may be downing it because we cannot communicate with it after all, in such situation (and many others) /// the other node would never receive this direct kill/down eager "gossip." We hope it will either receive the down via some means, or determine /// by itself that it should down itself. - internal static func shootTheOtherNodeAndCloseConnection(system: ActorSystem, targetNodeRemoteControl: AssociationRemoteControl) { + internal static func shootTheOtherNodeAndCloseConnection(system: ActorSystem, targetNodeAssociation: Association.AssociationState) { let log = system.log - let remoteNode = targetNodeRemoteControl.remoteNode + let remoteNode = targetNodeAssociation.remoteNode traceLog_Remote(system.cluster.node, "Finish terminate association [\(remoteNode)]: Shooting the other node a direct .gossip to down itself") // On purpose use the "raw" RemoteControl to send the message -- this way we avoid the association lookup (it may already be removed), // and directly hit the channel. It is also guaranteed that the message is flushed() before we close it in the next line. - let shootTheOtherNodePromise = system._eventLoopGroup.next().makePromise(of: Void.self) + let shootTheOtherNodePromise: EventLoopPromise = system._eventLoopGroup.next().makePromise(of: Void.self) let ripMessage = Envelope(payload: .message(ClusterShell.Message.inbound(.restInPeace(remoteNode, from: system.cluster.node)))) - targetNodeRemoteControl.sendUserMessage( + targetNodeAssociation.sendUserMessage( envelope: ripMessage, recipient: ._clusterShell(on: remoteNode), promise: shootTheOtherNodePromise @@ -166,33 +168,47 @@ internal class ClusterShell { shootTheOtherNodePromise.fail(TimeoutError(message: "Timed out writing final STONITH to \(remoteNode), should close forcefully.", timeout: .seconds(10))) // FIXME: same timeout but diff type } - shootTheOtherNodePromise.futureResult.flatMap { _ in + shootTheOtherNodePromise.futureResult.map { _ in // Only after the write has completed, we close the channel - targetNodeRemoteControl.closeChannel() + targetNodeAssociation.terminate(system) }.whenComplete { reason in log.trace("Closed connection with \(remoteNode): \(reason)") } } + /// For testing only. /// Safe to concurrently access by privileged internals. - internal var associationRemoteControls: [AssociationRemoteControl] { + internal var _testingOnly_associations: [Association.AssociationState] { self._associationsLock.withLock { - [AssociationRemoteControl](self._associationsRegistry.values) + [Association.AssociationState](self._activeAssociations.values) } } - /// To be invoked by cluster shell whenever an association is made; - /// The cache is used by remote actor refs to obtain means of sending messages into the pipeline, - /// without having to queue through the cluster shell's mailbox. - private func cacheAssociationRemoteControl(_ associationState: Association.AssociatedState) { - self._associationsLock.withLockVoid { - // TODO: or association ID rather than the remote id? - self._associationsRegistry[associationState.remoteNode] = associationState.makeRemoteControl() + /// For testing only. + internal func _associatedNodes() -> Set { + self._associationsLock.withLock { + Set(self._activeAssociations.keys) } } // ~~~~~~ END OF HERE BE DRAGONS, shared concurrently modified concurrent state ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ + // `_serializationPool` is only used when `start()` is invoked, and there it is set immediately as well + // any earlier access to the pool is a bug (in our library) and must be treated as such. + private var _serializationPool: SerializationPool? + internal var serializationPool: SerializationPool { + guard let pool = self._serializationPool else { + fatalError("BUG! Tried to access serializationPool on \(self) and it was nil! Please report this on the issue tracker.") + } + return pool + } + + // TODO: doc concurrency around this one (init in init) + private var _swimRef: SWIM.Ref? + + // TODO: doc concurrency around this one (init in init) + private var clusterEvents: EventStream! + // ==== ------------------------------------------------------------------------------------------------------------ // MARK: Cluster Shell, reference used for issuing commands to the cluster @@ -206,10 +222,11 @@ internal class ClusterShell { return it } - init() { + init(selfNode: UniqueNode) { + self.selfNode = selfNode self._associationsLock = Lock() - self._associationsRegistry = [:] - self._associationTombstones = [] + self._activeAssociations = [:] + self._associationTombstones = [:] // not enjoying this dance, but this way we can share the ClusterShell as the shell AND the container for the ref. // the single thing in the class it will modify is the associations registry, which we do to avoid actor queues when @@ -285,7 +302,7 @@ internal class ClusterShell { } internal enum InboundMessage { - case handshakeOffer(Wire.HandshakeOffer, channel: Channel, replyTo: EventLoopPromise) + case handshakeOffer(Wire.HandshakeOffer, channel: Channel, replyInto: EventLoopPromise) case handshakeAccepted(Wire.HandshakeAccept, channel: Channel) case handshakeRejected(Wire.HandshakeReject) case handshakeFailed(Node, Error) // TODO: remove? @@ -371,7 +388,9 @@ extension ClusterShell { let gossipControl = try ConvergentGossip.start( context, name: "\(ActorAddress._clusterGossip.name)", of: Cluster.Gossip.self, - notifyOnGossipRef: context.messageAdapter(from: Cluster.Gossip.self) { Optional.some(Message.gossipFromGossiper($0)) }, + notifyOnGossipRef: context.messageAdapter(from: Cluster.Gossip.self) { + Optional.some(Message.gossipFromGossiper($0)) + }, props: ._wellKnown ) @@ -442,7 +461,7 @@ extension ClusterShell { switch query { case .associatedNodes(let replyTo): - replyTo.tell(state.associatedNodes()) // TODO: we'll want to put this into some nicer message wrapper? + replyTo.tell(self._associatedNodes()) return .same case .currentMembership(let replyTo): replyTo.tell(state.membership) @@ -512,7 +531,9 @@ extension ClusterShell { "Local membership version is [.\(mergeDirective.causalRelation)] to incoming gossip; Merge resulted in \(mergeDirective.effectiveChanges.count) changes.", metadata: [ "tag": "membership", - "membership/changes": Logger.MetadataValue.array(mergeDirective.effectiveChanges.map { Logger.MetadataValue.stringConvertible($0) }), + "membership/changes": Logger.MetadataValue.array(mergeDirective.effectiveChanges.map { + Logger.MetadataValue.stringConvertible($0) + }), "gossip/incoming": "\(gossip)", "gossip/before": "\(beforeGossipMerge)", "gossip/now": "\(state.latestGossip)", @@ -591,18 +612,28 @@ extension ClusterShell { guard remoteNode != state.myselfNode.node else { state.log.debug("Ignoring attempt to handshake with myself; Could have been issued as confused attempt to handshake as induced by discovery via gossip?") replyTo?.tell(.failure(.init(node: remoteNode, message: "Would have attempted handshake with self node, aborted handshake."))) - return .same // TODO: could be drop + return .same } - if let existingAssociation = state.association(with: remoteNode) { + // if an association exists for any UniqueNode that this Node represents, we can use this and abort the handshake dance here + if let existingAssociation = self.getExistingAssociation(with: remoteNode) { state.log.debug("Attempted associating with already associated node: \(reflecting: remoteNode), existing association: [\(existingAssociation)]") - switch existingAssociation { - case .associated(let associationState): - replyTo?.tell(.success(associationState.remoteNode)) + switch existingAssociation.state { + case .associating: + () // ok, continue the association dance + case .associated: + // return , we've been successful already + replyTo?.tell(.success(existingAssociation.remoteNode)) + return .same case .tombstone: - replyTo?.tell(.failure(HandshakeConnectionError(node: remoteNode, message: "Existing association for \(remoteNode) is already a tombstone! Must not complete association."))) + replyTo?.tell(.failure( + HandshakeConnectionError( + node: existingAssociation.remoteNode.node, + message: "Existing association for \(existingAssociation.remoteNode) is already a tombstone! Must not complete association." + ) + )) + return .same } - return .same // TODO: could be drop } let whenHandshakeComplete = state.eventLoopGroup.next().makePromise(of: Wire.HandshakeResponse.self) @@ -644,6 +675,7 @@ extension ClusterShell { var state = state state.log.info("Extending handshake offer to \(initiated.remoteNode))") // TODO: log retry stats? + let offer: Wire.HandshakeOffer = initiated.makeOffer() self.tracelog(context, .send(to: initiated.remoteNode), message: offer) @@ -677,7 +709,8 @@ extension ClusterShell { /// Initial entry point for accepting a new connection; Potentially allocates new handshake state machine. internal func onHandshakeOffer( _ context: ActorContext, _ state: ClusterShellState, - _ offer: Wire.HandshakeOffer, channel: Channel, replyInto promise: EventLoopPromise + _ offer: Wire.HandshakeOffer, channel: Channel, + replyInto promise: EventLoopPromise ) -> Behavior { var state = state @@ -690,21 +723,36 @@ extension ClusterShell { // create and store association let directive = state.associate(context.system, completedHandshake, channel: channel) - let association = directive.association - self.cacheAssociationRemoteControl(association) + self.storeCompleteAssociation(directive) // send accept to other node let accept = completedHandshake.makeAccept() self.tracelog(context, .send(to: offer.from.node), message: accept) promise.succeed(.accept(accept)) - if directive.membershipChange.replaced != nil, - let removalDirective = directive.beingReplacedAssociationToTerminate { - state.log.warning("Tombstone association: \(reflecting: removalDirective.tombstone.remoteNode)") - self.finishTerminateAssociation(context.system, state: &state, removalDirective: removalDirective) + // TODO: This is a bit duplicated + // This association may mean that we've "replaced" a previously known node of the same host:port, + // In case of such replacement we must down and terminate the association of the previous node. + if let replacedMember = directive.membershipChange.replaced { + // the change was a replacement and thus we need to down the old member (same host:port as the new one), + // and terminate its association. + + state.log.info("Accepted handshake from [\(accept.from)] which replaces the previously known: \(reflecting: replacedMember).") + + // We MUST be careful to first terminate the association and then store the new one in 2) + self.terminateAssociation(context.system, state: &state, replacedMember.node) + + // By emitting these `change`s, we not only let anyone interested know about this, + // but we also enable the shell (or leadership) to update the leader if it needs changing. + // + // We MUST emit this `.down` before emitting the replacement's event + state.events.publish(.membershipChange(.init(member: replacedMember, toStatus: .down))) } - // TODO: try to pull off with receptionist the same dance + // publish any cluster events this association caused. + // As the new association is stored, any reactions to these events will use the right underlying connection + state.events.publish(.membershipChange(directive.membershipChange)) // TODO: need a test where a leader observes a replacement, and we ensure that it does not end up signalling up or removal twice? + self.tryIntroduceGossipPeer(context, state, change: directive.membershipChange) /// a new node joined, thus if we are the leader, we should perform leader tasks to potentially move it to .up @@ -791,44 +839,55 @@ extension ClusterShell { private func onHandshakeAccepted(_ context: ActorContext, _ state: ClusterShellState, _ accept: Wire.HandshakeAccept, channel: Channel) -> Behavior { var state = state // local copy for mutation - guard let completed = state.incomingHandshakeAccept(accept) else { - if state.associatedNodes().contains(accept.from) { + guard let handshakeCompleted = state.incomingHandshakeAccept(accept) else { + if self._associatedNodes().contains(accept.from) { // this seems to be a re-delivered accept, we already accepted association with this node. return .same - - // TODO: check tombstones as well } else { state.log.error("Illegal handshake accept received. No handshake was in progress with \(accept.from)") // TODO: tests and think this through more return .same } } - let directive = state.associate(context.system, completed, channel: channel) - self.cacheAssociationRemoteControl(directive.association) - state.log.debug("Associated with: \(reflecting: completed.remoteNode); Membership change: \(directive.membershipChange), resulting in: \(state.membership)") + // 1) Associate the new node + let directive = state.associate(context.system, handshakeCompleted, channel: channel) - self.tryIntroduceGossipPeer(context, state, change: directive.membershipChange) + // 1.1) This association may mean that we've "replaced" a previously known node of the same host:port, + // In case of such replacement we must down and terminate the association of the previous node. + if let replacedMember = directive.membershipChange.replaced { + // the change was a replacement and thus we need to down the old member (same host:port as the new one), + // and terminate its association. - // by emitting these `change`s, we not only let anyone interested know about this, - // but we also enable the shell (or leadership) to update the leader if it needs changing. - if directive.membershipChange.replaced != nil, - let replacedNodeRemovalDirective = directive.beingReplacedAssociationToTerminate { - state.log.warning("Tombstone association: \(reflecting: replacedNodeRemovalDirective.tombstone.remoteNode)") - // MUST be finishTerminate... and not terminate... because if we started a full terminateAssociation here - // we would terminate the _current_ association which was already removed by the associate() because - // it already _replaced_ some existing association (held in beingReplacedAssociation) - self.finishTerminateAssociation(context.system, state: &state, removalDirective: replacedNodeRemovalDirective) + state.log.info("Accepted handshake from [\(accept.from)] which replaces the previously known: \(reflecting: replacedMember).") + + // We MUST be careful to first terminate the association and then store the new one in 2) + self.terminateAssociation(context.system, state: &state, replacedMember.node) + + // By emitting these `change`s, we not only let anyone interested know about this, + // but we also enable the shell (or leadership) to update the leader if it needs changing. + // + // We MUST emit this `.down` before emitting the replacement's event + state.events.publish(.membershipChange(.init(member: replacedMember, toStatus: .down))) } - /// a new node joined, thus if we are the leader, we should perform leader tasks to potentially move it to .up + // 2) Store the (now completed) association first, as it may be immediately used by remote ActorRefs attempting to send to the remoteNode + self.storeCompleteAssociation(directive) // we MUST store it outside the state + state.log.debug("Associated with: \(reflecting: handshakeCompleted.remoteNode); Membership change: \(directive.membershipChange), resulting in: \(state.membership)") + + // 3) publish any cluster events this association caused. + // As the new association is stored, any reactions to these events will use the right underlying connection + state.events.publish(.membershipChange(directive.membershipChange)) // TODO: need a test where a leader observes a replacement, and we ensure that it does not end up signalling up or removal twice? + + self.tryIntroduceGossipPeer(context, state, change: directive.membershipChange) + + // 4) Since a new node joined, if we are the leader, we should perform leader tasks to potentially move it to .up let actions = state.collectLeaderActions() state = self.interpretLeaderActions(context.system, state, actions) - // TODO: return self.changedMembership which can do the publishing and publishing of metrics? we do it now in two places separately (incoming/outgoing accept) - /// only after leader (us, if we are one) performed its tasks, we update the metrics on membership (it might have modified membership) self.recordMetrics(context.system.metrics, membership: state.membership) - completed.whenCompleted?.succeed(.accept(completed.makeAccept())) + // 5) Finally, signal the handshake future that we've accepted, and become with ready state + handshakeCompleted.whenCompleted?.succeed(.accept(handshakeCompleted.makeAccept())) return self.ready(state: state) } @@ -1022,24 +1081,11 @@ extension ClusterShell { } // ==== ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - // Down(other node); + // Terminate association and Down the (other) node - guard let association = state.association(with: memberToDown.node.node) else { - context.log.warning("Received Down command for not associated node [\(reflecting: memberToDown.node.node)], ignoring.") - state = self.interpretLeaderActions(context.system, state, state.collectLeaderActions()) - return state - } - - switch association { - case .associated(let associated): - self.terminateAssociation(context.system, state: &state, associated) - state = self.interpretLeaderActions(context.system, state, state.collectLeaderActions()) - return state - case .tombstone: - state.log.warning("Attempted to .down already tombstoned association/node: [\(memberToDown)]") - state = self.interpretLeaderActions(context.system, state, state.collectLeaderActions()) - return state - } + self.terminateAssociation(context.system, state: &state, memberToDown.node) + state = self.interpretLeaderActions(context.system, state, state.collectLeaderActions()) + return state } } @@ -1048,6 +1094,7 @@ extension ClusterShell { extension ActorAddress { internal static let _clusterShell: ActorAddress = ActorPath._clusterShell.makeLocalAddress(incarnation: .wellKnown) + internal static func _clusterShell(on node: UniqueNode? = nil) -> ActorAddress { switch node { case .none: @@ -1058,6 +1105,7 @@ extension ActorAddress { } internal static let _clusterGossip: ActorAddress = ActorPath._clusterGossip.makeLocalAddress(incarnation: .wellKnown) + internal static func _clusterGossip(on node: UniqueNode? = nil) -> ActorAddress { switch node { case .none: diff --git a/Sources/DistributedActors/Cluster/ClusterShellState.swift b/Sources/DistributedActors/Cluster/ClusterShellState.swift index 280a87197..6070f68bd 100644 --- a/Sources/DistributedActors/Cluster/ClusterShellState.swift +++ b/Sources/DistributedActors/Cluster/ClusterShellState.swift @@ -56,7 +56,6 @@ internal struct ClusterShellState: ReadOnlyClusterState { let allocator: ByteBufferAllocator internal var _handshakes: [Node: HandshakeStateMachine.State] = [:] - private var _associations: [Node: Association.State] = [:] let gossipControl: ConvergentGossipControl @@ -109,23 +108,6 @@ internal struct ClusterShellState: ReadOnlyClusterState { self.channel = channel } - func association(with node: Node) -> Association.State? { - self._associations[node] - } - - func associatedNodes() -> Set { - var set: Set = .init(minimumCapacity: self._associations.count) - - for asm in self._associations.values { - switch asm { - case .associated(let state): set.insert(state.remoteNode) - case .tombstone: () - } - } - - return set - } - func handshakes() -> [HandshakeStateMachine.State] { self._handshakes.values.map { hsm -> HandshakeStateMachine.State in hsm @@ -154,10 +136,6 @@ extension ClusterShellState { return .inFlight(HandshakeStateMachine.InFlightState(state: self, whenCompleted: whenCompleted)) } - if let existingAssociation = self.association(with: remoteNode) { - fatalError("Beginning new handshake to [\(reflecting: remoteNode)], with already existing association: \(existingAssociation). Could this be a bug?") - } - let initiated = HandshakeStateMachine.InitiatedState( settings: self.settings, localNode: self.myselfNode, @@ -350,6 +328,7 @@ extension ClusterShellState { // based on the stored member let changeOption: Cluster.MembershipChange? = self.membership.applyMembershipChange(.init(member: .init(node: handshake.remoteNode, status: .joining))) ?? self.membership.uniqueMember(handshake.remoteNode).map { Cluster.MembershipChange(member: $0) } + guard let change = changeOption else { fatalError(""" Attempt to associate with \(reflecting: handshake.remoteNode) failed; It was neither a new node .joining, \ @@ -358,61 +337,16 @@ extension ClusterShellState { """) } - // Note: The following replace handling has to be done here - before we complete the association(!) - // As otherwise querying the associations by node would return the new one, leaving the old "replaced one" hanging (and channel not-closed). - // - // If the change is a replacement of a previously associated note, i.e. the remote node died, we didn't notice yet, - // but a new instance was started on the same host:port and now has reached out to us to associate. We need to eject - // the previous "replaced" node, and mark it as down, while at the same time accepting the association from the new node. - let removalDirective = self.removeAssociation(system, associatedNode: handshake.remoteNode) - if let replacedMember = change.replaced { - if removalDirective != nil { - self.log.warning("Node \(reflecting: handshake.remoteNode) joining OVER existing associated node \(reflecting: replacedMember.node) as its replacement. Severing ties with previous incarnation of node.") - // we are fairly certain the old node is dead now, since the new node is taking its place and has same address, - // thus the channel is most likely pointing to an "already-dead" connection; we close it to cut off clean. - self.events.publish(.membershipChange(.init(member: replacedMember, toStatus: .down))) - } else { - self.log.warning("Membership change indicated node replacement, yet no 'old' association to replace found. Continuing with association of \(reflecting: handshake.remoteNode)") - } - } - - self.events.publish(.membershipChange(change)) - - // Usual happy-path for an association; We associated a new node. - let association = Association.AssociatedState(fromCompleted: handshake, log: self.log, over: channel) - self._associations[handshake.remoteNode.node] = .associated(association) - - return AssociatedDirective(membershipChange: change, association: association, beingReplacedAssociationToTerminate: removalDirective) + return AssociatedDirective(membershipChange: change, handshake: handshake, channel: channel) } struct AssociatedDirective { let membershipChange: Cluster.MembershipChange - let association: Association.AssociatedState - - /// An association was replaced by the `membershipChange` and this "old" association must be terminated, pruned from caches, and tombstoned. - let beingReplacedAssociationToTerminate: RemoveAssociationDirective? + // let association: Association.AssociationState + let handshake: HandshakeStateMachine.CompletedState + let channel: Channel } - /// Performs only a removal of the association (if it is stored), and returns the tombstone for it. - /// It MAY happen that an association was already removed, e.g. in case of "replacement" joins. - mutating func removeAssociation(_ system: ActorSystem, associatedNode: UniqueNode) -> RemoveAssociationDirective? { - switch self._associations.removeValue(forKey: associatedNode.node) { - case .some(.associated(let associated)): - // if a value was NOT removed, we may have already replaced it with another association etc. - let tombstone = associated.makeTombstone(system: system) - return .init(removedAssociation: associated, tombstone: tombstone) - case .some(.tombstone(let tombstone)): - // TODO: this should rarely, if ever happen, why attempt to tombstone an existing tombstone, but people could issue many downs after all - return .init(removedAssociation: nil, tombstone: tombstone) - case .none: - return nil - } - } - - struct RemoveAssociationDirective { - let removedAssociation: Association.AssociatedState? - let tombstone: Association.TombstoneState - } } // ==== ---------------------------------------------------------------------------------------------------------------- diff --git a/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift b/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift index 300849efd..5ace78571 100644 --- a/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift +++ b/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift @@ -210,7 +210,7 @@ internal struct HandshakeStateMachine { // MARK: Handshake Completed /// State reached once we have received a `HandshakeAccepted` and are ready to create an association. - /// This state is used to unlock creating an Association. + /// This state is used to unlock creating a completed Association. internal struct CompletedState { let protocolVersion: Version var remoteNode: UniqueNode diff --git a/Sources/DistributedActors/Cluster/SWIM/SWIMInstance.swift b/Sources/DistributedActors/Cluster/SWIM/SWIMInstance.swift index d61ee4f03..313b37936 100644 --- a/Sources/DistributedActors/Cluster/SWIM/SWIMInstance.swift +++ b/Sources/DistributedActors/Cluster/SWIM/SWIMInstance.swift @@ -300,19 +300,19 @@ final class SWIMInstance { self._protocolPeriod } - /// Debug only. Actual suspicion timeout depends on number of suspicsions and calculated in `suspicionTimeout` + /// Debug only. Actual suspicion timeout depends on number of suspicions and calculated in `suspicionTimeout` /// This will only show current estimate of how many intervals should pass before suspicion is reached. May change when more data is coming var timeoutSuspectsBeforePeriodMax: Int64 { self.settings.lifeguard.suspicionTimeoutMax.nanoseconds / self.dynamicLHMProtocolInterval.nanoseconds + 1 } - /// Debug only. Actual suspicion timeout depends on number of suspicsions and calculated in `suspicionTimeout` + /// Debug only. Actual suspicion timeout depends on number of suspicions and calculated in `suspicionTimeout` /// This will only show current estimate of how many intervals should pass before suspicion is reached. May change when more data is coming var timeoutSuspectsBeforePeriodMin: Int64 { self.settings.lifeguard.suspicionTimeoutMin.nanoseconds / self.dynamicLHMProtocolInterval.nanoseconds + 1 } - /// The forumla is taken from Lifeguard whitepaper https://arxiv.org/abs/1707.00788 + /// The suspicion timeout is calculated as defined in Lifeguard Section IV.B https://arxiv.org/abs/1707.00788 /// According to it, suspicion timeout is logarithmically decaying from `suspicionTimeoutPeriodsMax` to `suspicionTimeoutPeriodsMin` /// depending on a number of suspicion confirmations. /// @@ -340,6 +340,7 @@ final class SWIMInstance { return max(minTimeout, .nanoseconds(maxTimeout.nanoseconds - Int64(round(Double(maxTimeout.nanoseconds - minTimeout.nanoseconds) * (log2(Double(suspectedByCount + 1)) / log2(Double(self.settings.lifeguard.maxIndependentSuspicions + 1))))))) } + /// Checks if a deadline is expired (relating to current time). func isExpired(deadline: Int64) -> Bool { deadline < self.timeSourceNanos() } @@ -374,7 +375,7 @@ final class SWIMInstance { self.members.mapValues { $0.status } } - /// Lists all suspect members, including myself if suspect. + /// Lists all suspect members. var suspects: SWIM.Members { self.members .lazy @@ -476,6 +477,7 @@ extension SWIM.Instance { case .failure: // missed pingReq's nack may indicate a problem with local health self.adjustLHMultiplier(.probeWithMissedNack) + switch lastKnownStatus { case .alive(let incarnation), .suspect(let incarnation, _): switch self.mark(member, as: self.makeSuspicion(incarnation: incarnation)) { diff --git a/Sources/DistributedActors/Cluster/SWIM/SWIMShell.swift b/Sources/DistributedActors/Cluster/SWIM/SWIMShell.swift index bd573ace4..b71722815 100644 --- a/Sources/DistributedActors/Cluster/SWIM/SWIMShell.swift +++ b/Sources/DistributedActors/Cluster/SWIM/SWIMShell.swift @@ -383,7 +383,6 @@ internal struct SWIMShell { for suspect in self.swim.suspects { if case .suspect(_, let suspectedBy) = suspect.status { - // TODO: push more of logic into SWIM instance, the calculating let suspicionTimeout = self.swim.suspicionTimeout(suspectedByCount: suspectedBy.count) context.log.trace( "Checking suspicion timeout for: \(suspect)...", @@ -396,7 +395,8 @@ internal struct SWIMShell { // proceed with suspicion escalation to .unreachable if the timeout period has been exceeded // We don't use Deadline because tests can override TimeSource - guard let startTime = suspect.suspicionStartedAt, self.swim.isExpired(deadline: startTime + suspicionTimeout.nanoseconds) else { + guard let startTime = suspect.suspicionStartedAt, + self.swim.isExpired(deadline: startTime + suspicionTimeout.nanoseconds) else { continue // skip, this suspect is not timed-out yet } @@ -526,16 +526,9 @@ internal struct SWIMShell { return } - let associationState = clusterShell.associationRemoteControl(with: remoteNode) + let associationState = clusterShell.getEnsureAssociation(with: remoteNode) switch associationState { - case .unknown: - // This may mean that we noticed an actor on a not yet associated node in the SWIM gossip, - // and need to ensure we connect to that node in order to be able to monitor it; thus we need to kick off a handshake with that node. - // - // Note that we DO know the remote's `UniqueNode`, putting us in the interesting position that we know exactly which incarnation of a - // node we intend to talk to -- unlike a plain "join node" command. - () // continue - case .associated(let control): + case .association(let control): continueWithAssociation(.success(control.remoteNode)) case .tombstone: let msg = "Association target node is already .tombstoned, not associating. Node \(reflecting: remoteNode) likely to be removed from gossip shortly." diff --git a/Sources/DistributedActors/Cluster/SystemMessages+Redelivery.swift b/Sources/DistributedActors/Cluster/SystemMessages+Redelivery.swift index a54fa8ca8..b2fcca657 100644 --- a/Sources/DistributedActors/Cluster/SystemMessages+Redelivery.swift +++ b/Sources/DistributedActors/Cluster/SystemMessages+Redelivery.swift @@ -252,6 +252,7 @@ internal final class OutboundSystemMessageRedelivery { // ./' ./' ./' Never gonna give you up, never gonna let you down ./' ./' ./' // we currently never give up + // FIXME: implement giving up reconnecting // if self.redeliveryTicksSinceLastACK > self.maxRedeliveryTicksWithoutACK { // return GiveUpRedeliveringSystemMessagesError() // } else { @@ -259,7 +260,7 @@ internal final class OutboundSystemMessageRedelivery { // } } - func onReconnected(newAssociationID: Association.AssociatedState) { + func onReconnected(newAssociationID: Association.AssociationState) { // TODO: redeliver everything } diff --git a/Sources/DistributedActors/Cluster/Transport/ActorRef+RemotePersonality.swift b/Sources/DistributedActors/Cluster/Transport/ActorRef+RemotePersonality.swift deleted file mode 100644 index 6cc33da1f..000000000 --- a/Sources/DistributedActors/Cluster/Transport/ActorRef+RemotePersonality.swift +++ /dev/null @@ -1,148 +0,0 @@ -//===----------------------------------------------------------------------===// -// -// This source file is part of the Swift Distributed Actors open source project -// -// Copyright (c) 2018-2019 Apple Inc. and the Swift Distributed Actors project authors -// Licensed under Apache License v2.0 -// -// See LICENSE.txt for license information -// See CONTRIBUTORS.md for the list of Swift Distributed Actors project authors -// -// SPDX-License-Identifier: Apache-2.0 -// -//===----------------------------------------------------------------------===// - -/// :nodoc: INTERNAL API: May change without any prior notice. -/// -/// Represents a reference to a remote actor. -/// -/// By owning a reference itself, no guarantees are given if the actor exists or even existed. -/// These guarantees are provided by ways of how this reference was obtained, e.g. if it was obtained -/// by being sent from a remote note, one can safely assume that the actor _existed_, however nothing -/// is clear about its current lifecycle state (it may have already terminated the moment the message was sent, -/// or even before then). To obtain lifecycle status of this actor the usual strategy of watching it needs to be employed. -// TODO: reimplement as CellDelegate as it shall become simply another transport? -public final class RemotePersonality { - let address: ActorAddress - - let deadLetters: ActorRef - - // Implementation notes: - // - // Goal: we want to hand out the ref as soon as possible and then if someone uses it they may pay the for accessing - // the associations there; - // - // Problem: - // - obtaining an association will hit a lock currently, since it is stored in this associations map - // - even if we do a concurrent map, it still is more expensive - // - // Observations: - // - we only need the association for the first send -- we can then hit the shared data-structure, and cache the association / remote control here - // - not all actor refs will be send to perhaps, so we can avoid hitting the shared structure at all sometimes - // - // The structure of the shell is such that the only thing that is a field in the class is this associations / remote controls map, - // which refs access. all other state is not accessible by anyone else since it is hidden in the actor itself. - - // TODO: -// // Access only via `self.remoteControl` -// private var _cachedAssociationRemoteControl: AssociationRemoteControl? - - let clusterShell: ClusterShell - let system: ActorSystem // TODO: maybe don't need to store it and access via clusterShell? - - @usableFromInline - internal var instrumentation: ActorInstrumentation! - - init(shell: ClusterShell, address: ActorAddress, system: ActorSystem) { - assert(address.isRemote, "RemoteActorRef MUST be remote. ActorAddress was: \(String(reflecting: address))") - self.address = address - self.clusterShell = shell - self.deadLetters = system.deadLetters.adapted() - self.system = system - self.instrumentation = system.settings.instrumentation.makeActorInstrumentation(self, address) - } - - @usableFromInline - func sendUserMessage(_ message: Message, file: String = #file, line: UInt = #line) { - traceLog_Cell("RemoteActorRef(\(self.address)) sendUserMessage: \(message)") - if let remoteControl = self.remoteControl { - // TODO: optionally carry file/line? - self.instrumentation.actorTold(message: message, from: nil) - remoteControl.sendUserMessage(envelope: Envelope(payload: .message(message)), recipient: self.address) - } else { - self.system.log.warning("No remote control, while sending to: \(self.address)") - self.system.personalDeadLetters(recipient: self.address).adapted().tell(message, file: file, line: line) - } - } - -// @usableFromInline -// func sendUserMessage(_ message: Message, file: String = #file, line: UInt = #line) { -// self.sendUserMessage(message, messageTypeForSerialization: Message.self, file: file, line: line) -// } -// -// /// TRICK! By forcing the `messageTypeForSerialization` EXPLICITLY we are able to override what would otherwise be the Message type -// /// which is used to store the identifier of the message type, by which we perform serializer lookups. Using this, we are able to -// /// send messages with a specific sub-class, rather than the generic `Message` type which e.g. could be some protocol -- and thus would never -// /// have a serializer associated with it. -// /// -// // TODO: all this can likely be simplified when/if we go for string type manifests and simplify serialization. -// @usableFromInline -// func sendUserMessage(_ message: Message, messageTypeForSerialization messageType: MessageTypeForSerialization.Type, file: String = #file, line: UInt = #line) { -// traceLog_Cell("RemoteActorRef(\(self.address)) sendUserMessage: \(message) (as \(messageType))") -// if let remoteControl = self.remoteControl { -// // TODO: optionally carry file/line? -// self.instrumentation.actorTold(message: message, from: nil) -// remoteControl.sendUserMessage(type: messageType, envelope: Envelope(payload: .message(message)), recipient: self.address) -// } else { -// self.system.log.warning("No remote control, while sending to: \(self.address)") -// self.system.personalDeadLetters(recipient: self.address).adapted().tell(message, file: file, line: line) -// } -// } - - @usableFromInline - func sendSystemMessage(_ message: _SystemMessage, file: String = #file, line: UInt = #line) { - traceLog_Cell("RemoteActorRef(\(self.address)) sendSystemMessage: \(message)") - // TODO: in case we'd get a new connection the redeliveries must remain... so we always need to poll for the remotecontrol from association? - // the association would keep the buffers? - if let remoteControl = self.remoteControl { - self.instrumentation.actorTold(message: message, from: nil) - remoteControl.sendSystemMessage(message, recipient: self.address) - } else { - self.deadLetters.adapted().tell(message, file: file, line: line) - } - } - - // FIXME: The test_singletonByClusterLeadership_stashMessagesIfNoLeader exposes that we sometimes need to spin here!!! This is very bad, investigate - // FIXME: https://github.com/apple/swift-distributed-actors/issues/382 - private var remoteControl: AssociationRemoteControl? { - guard let remoteAddress = self.address.node else { - fatalError("Attempted to access association remote control yet ref has no address! This should never happen and is a bug.") - } - - // FIXME: this is a hack/workaround, see https://github.com/apple/swift-distributed-actors/issues/383 - let maxWorkaroundSpins = 500 - for spinNr in 1 ... maxWorkaroundSpins { - switch self.clusterShell.associationRemoteControl(with: remoteAddress) { - case .unknown: - // keep spinning... - Thread.sleep(.milliseconds(50)) - - case .associated(let remoteControl): - if spinNr > 1 { - self.system.log.notice("FIXME: Workaround, ActorRef's RemotePersonality had to spin \(spinNr) times to obtain remoteControl to send message to \(self.address)") - } - // self._cachedAssociationRemoteControl = remoteControl // TODO: atomically cache a remote control? - return remoteControl - - case .tombstone: - return nil - } - } - - return nil - } - - func _unsafeAssumeCast(to: NewMessage.Type) -> RemotePersonality { - RemotePersonality(shell: self.clusterShell, address: self.address, system: self.system) - } -} diff --git a/Sources/DistributedActors/Cluster/Transport/RemoteClusterActorPersonality.swift b/Sources/DistributedActors/Cluster/Transport/RemoteClusterActorPersonality.swift new file mode 100644 index 000000000..5c3c2b059 --- /dev/null +++ b/Sources/DistributedActors/Cluster/Transport/RemoteClusterActorPersonality.swift @@ -0,0 +1,114 @@ +//===----------------------------------------------------------------------===// +// +// This source file is part of the Swift Distributed Actors open source project +// +// Copyright (c) 2018-2019 Apple Inc. and the Swift Distributed Actors project authors +// Licensed under Apache License v2.0 +// +// See LICENSE.txt for license information +// See CONTRIBUTORS.md for the list of Swift Distributed Actors project authors +// +// SPDX-License-Identifier: Apache-2.0 +// +//===----------------------------------------------------------------------===// + +/// :nodoc: INTERNAL API: May change without any prior notice. +/// +/// Represents a reference to a remote actor. +/// +/// By owning a reference itself, no guarantees are given if the actor exists or even existed. +/// These guarantees are provided by ways of how this reference was obtained, e.g. if it was obtained +/// by being sent from a remote note, one can safely assume that the actor _existed_, however nothing +/// is clear about its current lifecycle state (it may have already terminated the moment the message was sent, +/// or even before then). To obtain lifecycle status of this actor the usual strategy of watching it needs to be employed. +// TODO: reimplement as CellDelegate as it shall become simply another transport? +public final class RemoteClusterActorPersonality { + let address: ActorAddress + + let clusterShell: ClusterShell + let system: ActorSystem // TODO: maybe don't need to store it and access via clusterShell? + + var deadLetters: ActorRef { + self.system.personalDeadLetters(recipient: self.address) + } + + // // Implementation notes: + // // + // // Goal: we want to hand out the ref as soon as possible and then if someone uses it they may pay the for accessing + // // the associations there; + // // + // // Problem: + // // - obtaining an association will hit a lock currently, since it is stored in this associations map + // // - even if we do a concurrent map, it still is more expensive + // // + // // Observations: + // // - we only need the association for the first send -- we can then hit the shared data-structure, and cache the association / remote control here + // // - not all actor refs will be send to perhaps, so we can avoid hitting the shared structure at all sometimes + // // + // // The structure of the shell is such that the only thing that is a field in the class is this associations / remote controls map, + // // which refs access. all other state is not accessible by anyone else since it is hidden in the actor itself. + // + // TODO: once we can depend on Swift's Atomics, this could use the UnsafeAtomicLazyReference to easily cache the association + // so we can avoid hitting the lock in the ClusterShell for each message send. + // private var _cachedAssociation: UnsafeAtomicLazyReference + + // TODO: move instrumentation into the transport? + @usableFromInline + internal var instrumentation: ActorInstrumentation! + + init(shell: ClusterShell, address: ActorAddress, system: ActorSystem) { + precondition(address.isRemote, "RemoteActorRef MUST be remote. ActorAddress was: \(String(reflecting: address))") + self.address = address + + self.clusterShell = shell + self.system = system + + self.instrumentation = system.settings.instrumentation.makeActorInstrumentation(self, address) // TODO: could be in association, per node + } + + @usableFromInline + func sendUserMessage(_ message: Message, file: String = #file, line: UInt = #line) { + traceLog_Cell("RemoteActorRef(\(self.address)) sendUserMessage: \(message)") + + switch self.association { + case .association(let association): + association.sendUserMessage(envelope: Envelope(payload: .message(message)), recipient: self.address) + self.instrumentation.actorTold(message: message, from: nil) + case .tombstone: + // TODO: metric for dead letter: self.instrumentation.deadLetter(message: message, from: nil) + self.deadLetters.tell(DeadLetter(message, recipient: self.address, sentAtFile: file, sentAtLine: line)) + } + } + + @usableFromInline + func sendSystemMessage(_ message: _SystemMessage, file: String = #file, line: UInt = #line) { + traceLog_Cell("RemoteActorRef(\(self.address)) sendSystemMessage: \(message)") + + // TODO: in case we'd get a new connection the redeliveries must remain... so we always need to poll for the remotecontrol from association? the association would keep the buffers? + // TODO: would this mean that we cannot implement re-delivery inside the NIO layer as we do today? + switch self.association { + case .association(let association): + association.sendSystemMessage(message, recipient: self.address) + self.instrumentation.actorTold(message: message, from: nil) + case .tombstone: + // TODO: metric for dead letter: self.instrumentation.deadLetter(message: message, from: nil) + self.deadLetters.tell(DeadLetter(message, recipient: self.address, sentAtFile: file, sentAtLine: line)) + } + } + + private var association: ClusterShell.StoredAssociationState { + guard let remoteAddress = self.address.node else { + fatalError("Attempted to access association remote control yet ref has no address! This should never happen and is a bug. The ref was: \(self)") + } + + // TODO: once we have UnsafeAtomicLazyReference initialize it here: + // if let assoc = self._cachedAssociation.load() { return assoc } + // else { get from shell and store here } + + return self.clusterShell.getEnsureAssociation(with: remoteAddress) + } + + func _unsafeAssumeCast(to: NewMessage.Type) -> RemoteClusterActorPersonality { + RemoteClusterActorPersonality(shell: self.clusterShell, address: self.address, system: self.system) + } +} diff --git a/Sources/DistributedActors/Cluster/Transport/TransportPipelines.swift b/Sources/DistributedActors/Cluster/Transport/TransportPipelines.swift index 4d12ec9fb..fd83140dd 100644 --- a/Sources/DistributedActors/Cluster/Transport/TransportPipelines.swift +++ b/Sources/DistributedActors/Cluster/Transport/TransportPipelines.swift @@ -116,7 +116,7 @@ final class ReceivingHandshakeHandler: ChannelInboundHandler, RemovableChannelHa self.log.debug("Received handshake offer from: [\(offer.from)] with protocol version: [\(offer.version)]") let promise = context.eventLoop.makePromise(of: Wire.HandshakeResponse.self) - self.cluster.tell(.inbound(.handshakeOffer(offer, channel: context.channel, replyTo: promise))) + self.cluster.tell(.inbound(.handshakeOffer(offer, channel: context.channel, replyInto: promise))) promise.futureResult.whenComplete { res in switch res { diff --git a/Sources/DistributedActors/Receptionist.swift b/Sources/DistributedActors/Receptionist.swift index 55950e3e7..78bbc8afa 100644 --- a/Sources/DistributedActors/Receptionist.swift +++ b/Sources/DistributedActors/Receptionist.swift @@ -50,7 +50,7 @@ public enum Receptionist { internal func _unsafeAsActorRef(_ addressable: AddressableActorRef) -> ActorRef { if addressable.isRemote() { - let remoteWellTypedPersonality: RemotePersonality = addressable.ref._unsafeGetRemotePersonality(Message.self) + let remoteWellTypedPersonality: RemoteClusterActorPersonality = addressable.ref._unsafeGetRemotePersonality(Message.self) return ActorRef(.remote(remoteWellTypedPersonality)) } else { guard let ref = addressable.ref as? ActorRef else { diff --git a/Sources/DistributedActors/Refs+any.swift b/Sources/DistributedActors/Refs+any.swift index b8d4079c9..a7d8f25c0 100644 --- a/Sources/DistributedActors/Refs+any.swift +++ b/Sources/DistributedActors/Refs+any.swift @@ -110,12 +110,12 @@ extension AddressableActorRef: _ReceivesSystemMessages { self.ref._deserializeDeliver(messageBytes, using: manifest, on: pool, file: file, line: line) } - public func _unsafeGetRemotePersonality(_ type: M.Type = M.self) -> RemotePersonality { + public func _unsafeGetRemotePersonality(_ type: M.Type = M.self) -> RemoteClusterActorPersonality { self.ref._unsafeGetRemotePersonality(M.self) } } -internal extension RemotePersonality { +internal extension RemoteClusterActorPersonality { @usableFromInline func _tellUnsafe(_ message: Any, file: String = #file, line: UInt = #line) { guard let _message = message as? Message else { @@ -140,7 +140,7 @@ internal extension ActorRef { } @usableFromInline - var _unsafeUnwrapRemote: RemotePersonality { + var _unsafeUnwrapRemote: RemoteClusterActorPersonality { switch self.personality { case .remote(let remote): return remote default: fatalError("Illegal downcast attempt from \(String(reflecting: self)) to ActorRefWithCell. This is a Swift Distributed Actors bug, please report this on the issue tracker.") diff --git a/Sources/DistributedActors/Refs.swift b/Sources/DistributedActors/Refs.swift index 0d503985f..5c06391d2 100644 --- a/Sources/DistributedActors/Refs.swift +++ b/Sources/DistributedActors/Refs.swift @@ -30,7 +30,7 @@ public struct ActorRef: ReceivesMessages, _ReceivesSystem /// Adj. self-conscious: feeling undue awareness of oneself, one's appearance, or one's actions. public enum Personality { case cell(ActorCell) - case remote(RemotePersonality) + case remote(RemoteClusterActorPersonality) case adapter(AbstractAdapter) case guardian(Guardian) case delegate(CellDelegate) @@ -176,7 +176,7 @@ public protocol _ReceivesSystemMessages: Codable { ) /// :nodoc: INTERNAL API - func _unsafeGetRemotePersonality(_ type: M.Type) -> RemotePersonality + func _unsafeGetRemotePersonality(_ type: M.Type) -> RemoteClusterActorPersonality } extension _ReceivesSystemMessages { @@ -277,7 +277,7 @@ extension ActorRef { ) } - public func _unsafeGetRemotePersonality(_ type: M.Type = M.self) -> RemotePersonality { + public func _unsafeGetRemotePersonality(_ type: M.Type = M.self) -> RemoteClusterActorPersonality { switch self.personality { case .remote(let personality): return personality._unsafeAssumeCast(to: type) @@ -487,7 +487,7 @@ internal struct TheOneWhoHasNoParent: _ReceivesSystemMessages { // FIXME: fix th } @usableFromInline - internal func _unsafeGetRemotePersonality(_ type: M.Type = M.self) -> RemotePersonality { + internal func _unsafeGetRemotePersonality(_ type: M.Type = M.self) -> RemoteClusterActorPersonality { CDistributedActorsMailbox.sact_dump_backtrace() fatalError("The \(self.address) actor MUST NOT be interacted with directly!") } diff --git a/Sources/DistributedActorsTestKit/Cluster/ClusteredNodesTestBase.swift b/Sources/DistributedActorsTestKit/Cluster/ClusteredNodesTestBase.swift index 2749fda5d..afba83323 100644 --- a/Sources/DistributedActorsTestKit/Cluster/ClusteredNodesTestBase.swift +++ b/Sources/DistributedActorsTestKit/Cluster/ClusteredNodesTestBase.swift @@ -292,7 +292,7 @@ extension ClusteredNodesTestBase { } public func assertNotAssociated( - system: ActorSystem, expectAssociatedNode node: UniqueNode, + system: ActorSystem, node: UniqueNode, timeout: TimeAmount? = nil, interval: TimeAmount? = nil, verbose: Bool = false ) throws { diff --git a/Tests/DistributedActorsTests/ActorSystem+Testing.swift b/Tests/DistributedActorsTests/ActorSystem+Testing.swift index 678a2f3e0..3f0857642 100644 --- a/Tests/DistributedActorsTests/ActorSystem+Testing.swift +++ b/Tests/DistributedActorsTests/ActorSystem+Testing.swift @@ -41,6 +41,6 @@ extension ActorSystem { fatalError("system._cluster shell must be available, was the resolve invoked too early (before system startup completed)?") } let remoteAddress = ActorAddress(node: remote.settings.cluster.uniqueBindNode, path: ref.path, incarnation: ref.address.incarnation) - return ActorRef(.remote(RemotePersonality(shell: shell, address: remoteAddress, system: self))) + return ActorRef(.remote(RemoteClusterActorPersonality(shell: shell, address: remoteAddress, system: self))) } } diff --git a/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift b/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift index 52253ed29..461f46331 100644 --- a/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift +++ b/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift @@ -61,7 +61,7 @@ final class ClusterAssociationTests: ClusteredNodesTestBase { // ==== ------------------------------------------------------------------------------------------------------------ // MARK: Joining into existing cluster - // FIXME: unlock this test + // FIXME: unlock this test // revisit func fixme_association_sameAddressNodeJoin_shouldOverrideExistingNode() throws { try shouldNotThrow { let (first, second) = self.setUpPair() @@ -195,8 +195,8 @@ final class ClusterAssociationTests: ClusteredNodesTestBase { local.cluster.join(node: remote.cluster.node.node) - try assertNotAssociated(system: local, expectAssociatedNode: remote.cluster.node) - try assertNotAssociated(system: remote, expectAssociatedNode: local.cluster.node) + try assertNotAssociated(system: local, node: remote.cluster.node) + try assertNotAssociated(system: remote, node: local.cluster.node) } func test_handshake_shouldNotifyOnRejection() throws { @@ -209,8 +209,8 @@ final class ClusterAssociationTests: ClusteredNodesTestBase { local.cluster.ref.tell(.command(.handshakeWith(remote.cluster.node.node, replyTo: p.ref))) - try assertNotAssociated(system: local, expectAssociatedNode: remote.cluster.node) - try assertNotAssociated(system: remote, expectAssociatedNode: local.cluster.node) + try assertNotAssociated(system: local, node: remote.cluster.node) + try assertNotAssociated(system: remote, node: local.cluster.node) switch try p.expectMessage() { case ClusterShell.HandshakeResult.failure(let err): @@ -239,7 +239,7 @@ final class ClusterAssociationTests: ClusteredNodesTestBase { thirdSystem.cluster.join(node: local.cluster.node.node) try assertAssociated(local, withExactly: [remote.cluster.node, thirdSystem.settings.cluster.uniqueBindNode]) - local._cluster?.associationRemoteControls.count.shouldEqual(2) + local._cluster?._testingOnly_associations.count.shouldEqual(2) } // FIXME: once initiated, handshake seem to retry until they succeed, that seems @@ -263,6 +263,36 @@ final class ClusterAssociationTests: ClusteredNodesTestBase { } } + func test_sendingMessageToNotYetAssociatedNode_mustCauseAssociationAttempt() throws { + let first = self.setUpNode("first") + let second = self.setUpNode("second") + + // actor on `second` node + let p2 = self.testKit(second).spawnTestProbe(expecting: String.self) + let secondOne: ActorRef = try second.spawn("second-1", .receive { _, message in + p2.tell("Got:\(message)") + return .same + }) + var secondFullAddress = secondOne.address + secondFullAddress.node = second.cluster.node + + // we somehow obtained a ref to secondOne (on second node) without associating second yet + // e.g. another node sent us that ref; This must cause buffering of sends to second and an association to be created. + + let resolveContext = ResolveContext(address: secondFullAddress, system: first) + let ref = first._resolve(context: resolveContext) + + try assertNotAssociated(system: first, node: second.cluster.node) + try assertNotAssociated(system: second, node: first.cluster.node) + + ref.tell("Hello!") // will be buffered until associated, and then delivered + + try p2.expectMessage("Got:Hello!") + + try assertAssociated(first, withExactly: second.cluster.node) + try assertAssociated(second, withExactly: first.cluster.node) + } + // ==== ------------------------------------------------------------------------------------------------------------ // MARK: Change membership on Down detected diff --git a/Tests/DistributedActorsTests/Cluster/ClusterLeaderActionsClusteredTests.swift b/Tests/DistributedActorsTests/Cluster/ClusterLeaderActionsClusteredTests.swift index c0354fe9d..7e07a3839 100644 --- a/Tests/DistributedActorsTests/Cluster/ClusterLeaderActionsClusteredTests.swift +++ b/Tests/DistributedActorsTests/Cluster/ClusterLeaderActionsClusteredTests.swift @@ -257,21 +257,21 @@ final class ClusterLeaderActionsClusteredTests: ClusteredNodesTestBase { try shouldNotThrow { let first = self.setUpNode("first") { settings in settings.cluster.autoLeaderElection = .lowestReachable(minNumberOfMembers: 2) - settings.cluster.downingStrategy = .timeout(.init(downUnreachableMembersAfter: .milliseconds(300))) + settings.cluster.downingStrategy = .timeout(.init(downUnreachableMembersAfter: .milliseconds(200))) } let p1 = self.testKit(first).spawnTestProbe(expecting: Cluster.Event.self) first.cluster.events.subscribe(p1.ref) let second = self.setUpNode("second") { settings in settings.cluster.autoLeaderElection = .lowestReachable(minNumberOfMembers: 2) - settings.cluster.downingStrategy = .timeout(.init(downUnreachableMembersAfter: .milliseconds(300))) + settings.cluster.downingStrategy = .timeout(.init(downUnreachableMembersAfter: .milliseconds(200))) } let p2 = self.testKit(second).spawnTestProbe(expecting: Cluster.Event.self) second.cluster.events.subscribe(p2.ref) let third = self.setUpNode("third") { settings in settings.cluster.autoLeaderElection = .lowestReachable(minNumberOfMembers: 2) - settings.cluster.downingStrategy = .timeout(.init(downUnreachableMembersAfter: .milliseconds(300))) + settings.cluster.downingStrategy = .timeout(.init(downUnreachableMembersAfter: .milliseconds(200))) } let p3 = self.testKit(third).spawnTestProbe(expecting: Cluster.Event.self) third.cluster.events.subscribe(p3.ref) @@ -291,14 +291,14 @@ final class ClusterLeaderActionsClusteredTests: ClusteredNodesTestBase { // on the leader node, the other node noticed as up: let testKit = self.testKit(first) - try testKit.eventually(within: .seconds(5)) { + try testKit.eventually(within: .seconds(10)) { let event: Cluster.Event? = try p1.maybeExpectMessage() switch event { case .membershipChange(.init(node: second.cluster.node, fromStatus: .up, toStatus: .down)): () case let other: throw testKit.error("Expected `second` [ up] -> [ .down], on first node, was: \(other, orElse: "nil")") } } - try testKit.eventually(within: .seconds(5)) { + try testKit.eventually(within: .seconds(10)) { let event: Cluster.Event? = try p1.maybeExpectMessage() switch event { case .membershipChange(.init(node: second.cluster.node, fromStatus: .down, toStatus: .removed)): () diff --git a/Tests/DistributedActorsTests/Cluster/RemoteActorRefProviderTests.swift b/Tests/DistributedActorsTests/Cluster/RemoteActorRefProviderTests.swift index 1c86ea470..3ea79e3fe 100644 --- a/Tests/DistributedActorsTests/Cluster/RemoteActorRefProviderTests.swift +++ b/Tests/DistributedActorsTests/Cluster/RemoteActorRefProviderTests.swift @@ -24,8 +24,9 @@ final class RemoteActorRefProviderTests: ActorSystemTestBase { } } - let node = UniqueNode(systemName: "RemoteAssociationTests", host: "127.0.0.1", port: 9559, nid: NodeID(888_888)) - lazy var remoteNode = ActorAddress(node: node, path: try! ActorPath._user.appending("henry").appending("hacker"), incarnation: .random()) + let localNode = UniqueNode(systemName: "RemoteAssociationTests", host: "127.0.0.1", port: 7111, nid: NodeID(777_777)) + let remoteNode = UniqueNode(systemName: "RemoteAssociationTests", host: "127.0.0.1", port: 9559, nid: NodeID(888_888)) + lazy var remoteAddress = ActorAddress(node: remoteNode, path: try! ActorPath._user.appending("henry").appending("hacker"), incarnation: .random()) // ==== ---------------------------------------------------------------------------------------------------------------- // MARK: Properly resolve @@ -36,7 +37,7 @@ final class RemoteActorRefProviderTests: ActorSystemTestBase { let guardian = Guardian(parent: theOne, name: "user", system: system) let localProvider = LocalActorRefProvider(root: guardian) - let clusterShell = ClusterShell() + let clusterShell = ClusterShell(selfNode: self.localNode) let provider = RemoteActorRefProvider(settings: system.settings, cluster: clusterShell, localProvider: localProvider) let node = UniqueNode(node: .init(systemName: "system", host: "3.3.3.3", port: 2322), nid: .random()) diff --git a/Tests/DistributedActorsTests/Cluster/ShootTheOtherNodeClusteredTests.swift b/Tests/DistributedActorsTests/Cluster/ShootTheOtherNodeClusteredTests.swift index de8215ea6..de22f7a46 100644 --- a/Tests/DistributedActorsTests/Cluster/ShootTheOtherNodeClusteredTests.swift +++ b/Tests/DistributedActorsTests/Cluster/ShootTheOtherNodeClusteredTests.swift @@ -34,12 +34,12 @@ final class ShootTheOtherNodeClusteredTests: ClusteredNodesTestBase { // also assures they are associated try self.joinNodes(node: local, with: remote, ensureWithin: .seconds(5), ensureMembers: .up) - let remoteAssociationControlState0 = local._cluster!.associationRemoteControl(with: remote.cluster.node) - guard case ClusterShell.AssociationRemoteControlState.associated(let remoteControl0) = remoteAssociationControlState0 else { + let remoteAssociationControlState0 = local._cluster!.getEnsureAssociation(with: remote.cluster.node) + guard case ClusterShell.StoredAssociationState.association(let remoteControl0) = remoteAssociationControlState0 else { throw Boom("Expected the association to exist for \(remote.cluster.node)") } - ClusterShell.shootTheOtherNodeAndCloseConnection(system: local, targetNodeRemoteControl: remoteControl0) + ClusterShell.shootTheOtherNodeAndCloseConnection(system: local, targetNodeAssociation: remoteControl0) // the remote should get the "shot" and become down asap try self.testKit(local).eventually(within: .seconds(3)) { From 0195ee88226ed76a971259095691d89323b0644d Mon Sep 17 00:00:00 2001 From: Konrad `ktoso` Malawski Date: Fri, 24 Apr 2020 16:44:32 +0900 Subject: [PATCH 02/18] =swim log also when suspision comes in from incoming gossip --- .../DistributedActors/Cluster/ClusterShellState.swift | 1 - .../DistributedActors/Cluster/SWIM/SWIMInstance.swift | 11 ++++++++++- .../DistributedActors/Cluster/SWIM/SWIMShell.swift | 2 +- 3 files changed, 11 insertions(+), 3 deletions(-) diff --git a/Sources/DistributedActors/Cluster/ClusterShellState.swift b/Sources/DistributedActors/Cluster/ClusterShellState.swift index 6070f68bd..8ed22ac26 100644 --- a/Sources/DistributedActors/Cluster/ClusterShellState.swift +++ b/Sources/DistributedActors/Cluster/ClusterShellState.swift @@ -346,7 +346,6 @@ extension ClusterShellState { let handshake: HandshakeStateMachine.CompletedState let channel: Channel } - } // ==== ---------------------------------------------------------------------------------------------------------------- diff --git a/Sources/DistributedActors/Cluster/SWIM/SWIMInstance.swift b/Sources/DistributedActors/Cluster/SWIM/SWIMInstance.swift index 313b37936..cac2ff9c3 100644 --- a/Sources/DistributedActors/Cluster/SWIM/SWIMInstance.swift +++ b/Sources/DistributedActors/Cluster/SWIM/SWIMInstance.swift @@ -602,7 +602,16 @@ extension SWIM.Instance { case .applied(let previousStatus, let currentStatus): var member = member member.status = currentStatus - return .applied(change: .init(fromStatus: previousStatus, member: member)) + if currentStatus.isSuspect, previousStatus?.isAlive ?? false { + return .applied( + change: .init(fromStatus: previousStatus, member: member), + level: .debug, + message: "Member [\(member.ref.address.node, orElse: "")] marked as suspect, via incoming gossip" + ) + } else { + return .applied(change: .init(fromStatus: previousStatus, member: member)) + } + case .ignoredDueToOlderStatus(let currentStatus): return .ignored( level: .trace, diff --git a/Sources/DistributedActors/Cluster/SWIM/SWIMShell.swift b/Sources/DistributedActors/Cluster/SWIM/SWIMShell.swift index b71722815..b3cf63138 100644 --- a/Sources/DistributedActors/Cluster/SWIM/SWIMShell.swift +++ b/Sources/DistributedActors/Cluster/SWIM/SWIMShell.swift @@ -396,7 +396,7 @@ internal struct SWIMShell { // proceed with suspicion escalation to .unreachable if the timeout period has been exceeded // We don't use Deadline because tests can override TimeSource guard let startTime = suspect.suspicionStartedAt, - self.swim.isExpired(deadline: startTime + suspicionTimeout.nanoseconds) else { + self.swim.isExpired(deadline: startTime + suspicionTimeout.nanoseconds) else { continue // skip, this suspect is not timed-out yet } From 0c66985d0f42b420454d0d822cc6b4a075819265 Mon Sep 17 00:00:00 2001 From: Konrad `ktoso` Malawski Date: Fri, 24 Apr 2020 17:17:49 +0900 Subject: [PATCH 03/18] =handshake no reason to crash, simply reject incoming unexpected handshake --- .../Cluster/Association.swift | 2 +- .../Cluster/ClusterShell.swift | 2 +- .../Cluster/ClusterShellState.swift | 39 ++++++++++--------- 3 files changed, 22 insertions(+), 21 deletions(-) diff --git a/Sources/DistributedActors/Cluster/Association.swift b/Sources/DistributedActors/Cluster/Association.swift index f4afd5c2d..98b4f5169 100644 --- a/Sources/DistributedActors/Cluster/Association.swift +++ b/Sources/DistributedActors/Cluster/Association.swift @@ -33,7 +33,7 @@ import NIO /// A completed ("associated") `Association` can ONLY be obtained by successfully completing a `HandshakeStateMachine` dance, /// as only the handshake can ensure that the other side is also an actor node that is able and willing to communicate with us. struct Association { - // TODO: reword all docs; association from now on means "from init until valid and reconnections as well" + final class AssociationState: CustomStringConvertible { // TODO: Terrible lock which we want to get rid of; it means that every remote send has to content against all other sends about getting this ref // and the only reason is really because the off chance case in which we have to make an Association earlier than we have the handshake completed (i.e. we send to a ref that is not yet associated) diff --git a/Sources/DistributedActors/Cluster/ClusterShell.swift b/Sources/DistributedActors/Cluster/ClusterShell.swift index 0bc575777..e30782b20 100644 --- a/Sources/DistributedActors/Cluster/ClusterShell.swift +++ b/Sources/DistributedActors/Cluster/ClusterShell.swift @@ -844,7 +844,7 @@ extension ClusterShell { // this seems to be a re-delivered accept, we already accepted association with this node. return .same } else { - state.log.error("Illegal handshake accept received. No handshake was in progress with \(accept.from)") // TODO: tests and think this through more + state.log.error("Unexpected handshake accept received: [\(accept)]. No handshake was in progress with \(accept.from)") return .same } } diff --git a/Sources/DistributedActors/Cluster/ClusterShellState.swift b/Sources/DistributedActors/Cluster/ClusterShellState.swift index 8ed22ac26..ba7c08058 100644 --- a/Sources/DistributedActors/Cluster/ClusterShellState.swift +++ b/Sources/DistributedActors/Cluster/ClusterShellState.swift @@ -289,26 +289,27 @@ extension ClusterShellState { case abortDueToConcurrentHandshake } - mutating func incomingHandshakeAccept(_ accept: Wire.HandshakeAccept) -> HandshakeStateMachine.CompletedState? { // TODO: return directives to act on - if let inProgressHandshake = self._handshakes[accept.from.node] { - switch inProgressHandshake { - case .initiated(let hsm): - let completed = HandshakeStateMachine.CompletedState(fromInitiated: hsm, remoteNode: accept.from) - return completed - case .wasOfferedHandshake: - // TODO: model the states to express this can not happen // there is a client side state machine and a server side one - self.log.warning("Received accept but state machine is in WAS OFFERED state. This should be impossible.") - return nil - case .completed: - // TODO: validate if it is for the same UID or not, if not, we may be in trouble? - self.log.warning("Received handshake Accept for already completed handshake. This should not happen.") - return nil - case .inFlight: - fatalError("An in-flight marker state should never be stored, yet was encountered in \(#function)") - } - } else { + mutating func incomingHandshakeAccept(_ accept: Wire.HandshakeAccept) -> HandshakeStateMachine.CompletedState? { + guard let inProgressHandshake = self._handshakes[accept.from.node] else { // TODO: what if node that sent handshake, has already terminated -- would we have removed the in progress handshake already causing this? - fatalError("Accept incoming [\(accept)] for handshake which was not in progress! On node: \(self.myselfNode), cluster shell state: \(self), membership: \(self.membership)") // TODO: model differently + // fatalError("Accept incoming [\(accept)] for handshake which was not in progress! On node: \(self.myselfNode), cluster shell state: \(self), membership: \(self.membership)") // TODO: model differently + return nil + } + + switch inProgressHandshake { + case .initiated(let hsm): + let completed = HandshakeStateMachine.CompletedState(fromInitiated: hsm, remoteNode: accept.from) + return completed + case .wasOfferedHandshake: + // TODO: model the states to express this can not happen // there is a client side state machine and a server side one + self.log.warning("Received accept but state machine is in WAS OFFERED state. This should be impossible.") + return nil + case .completed: + // TODO: validate if it is for the same UID or not, if not, we may be in trouble? + self.log.warning("Received handshake Accept for already completed handshake. This should not happen.") + return nil + case .inFlight: + fatalError("An in-flight marker state should never be stored, yet was encountered in \(#function)") } } From 24c22a0a7f6a515a3641acc086d7ef0540586ec9 Mon Sep 17 00:00:00 2001 From: Konrad `ktoso` Malawski Date: Sat, 25 Apr 2020 12:29:55 +0900 Subject: [PATCH 04/18] =cluster cleanups, swim logging and timeouts in tests --- .../it_XPCActorable_echo_service/main.swift | 2 +- .../main.swift | 4 +- Samples/Sources/SampleCluster/main.swift | 2 +- .../Cluster/Association.swift | 161 +++++++++--------- .../Cluster/Cluster+Gossip.swift | 4 +- .../Cluster/ClusterSettings.swift | 4 +- .../Cluster/ClusterShell.swift | 18 +- .../Cluster/ClusterShellState.swift | 2 +- .../Cluster/SWIM/SWIMInstance.swift | 8 +- .../Cluster/SWIM/SWIMSettings.swift | 147 +++++++++------- .../Cluster/SWIM/SWIMShell.swift | 4 +- .../Cluster/SystemMessages+Redelivery.swift | 2 +- Sources/DistributedActors/Receptionist.swift | 2 - ...olations.swift => String+Extensions.swift} | 10 ++ Sources/DistributedActors/Time.swift | 2 +- .../ActorContextReceptionistTests.swift | 2 +- .../Protobuf/CRDT+SerializationTests.swift | 38 ++--- .../CRDTEnvelope+SerializationTests.swift | 7 +- .../CRDTReplication+SerializationTests.swift | 2 +- .../VersionVector+SerializationTests.swift | 6 +- .../Cluster/AssociationClusteredTests.swift | 2 +- .../ClusterLeaderActionsClusteredTests.swift | 20 ++- .../Cluster/RemotingTLSClusteredTests.swift | 6 +- .../SWIM/SWIMShellClusteredTests.swift | 62 +++++-- .../NodeDeathWatcherTests.swift | 4 +- .../SerializationTests.swift | 2 +- 26 files changed, 299 insertions(+), 224 deletions(-) rename Sources/DistributedActors/{CustomStringInterpolations.swift => String+Extensions.swift} (84%) diff --git a/IntegrationTests/tests_03_xpc_actorable/it_XPCActorable_echo_service/main.swift b/IntegrationTests/tests_03_xpc_actorable/it_XPCActorable_echo_service/main.swift index f9e8f0098..86b95532b 100644 --- a/IntegrationTests/tests_03_xpc_actorable/it_XPCActorable_echo_service/main.swift +++ b/IntegrationTests/tests_03_xpc_actorable/it_XPCActorable_echo_service/main.swift @@ -24,7 +24,7 @@ try! _file.append("service starting...\n") let system = ActorSystem("it_XPCActorable_echo_service") { settings in settings.transports += .xpcService - settings.cluster.swim.failureDetector.pingTimeout = .seconds(3) + settings.cluster.swim.pingTimeout = .seconds(3) // settings.serialization.register(GeneratedActor.Messages.XPCEchoServiceProtocol.self, underId: 10001) // settings.serialization.register(XPCEchoService.Message.self, underId: 10002) diff --git a/IntegrationTests/tests_04_cluster/it_Clustered_swim_suspension_reachability/main.swift b/IntegrationTests/tests_04_cluster/it_Clustered_swim_suspension_reachability/main.swift index 935f16c53..28b26657b 100644 --- a/IntegrationTests/tests_04_cluster/it_Clustered_swim_suspension_reachability/main.swift +++ b/IntegrationTests/tests_04_cluster/it_Clustered_swim_suspension_reachability/main.swift @@ -31,10 +31,10 @@ let system = ActorSystem("System") { settings in settings.cluster.enabled = true settings.cluster.bindPort = Int(args[0])! + settings.cluster.swim.probeInterval = .milliseconds(300) + settings.cluster.swim.pingTimeout = .milliseconds(100) settings.cluster.swim.lifeguard.suspicionTimeoutMin = .seconds(1) settings.cluster.swim.lifeguard.suspicionTimeoutMax = .seconds(1) - settings.cluster.swim.failureDetector.pingTimeout = .milliseconds(100) - settings.cluster.swim.failureDetector.probeInterval = .milliseconds(300) settings.cluster.autoLeaderElection = .lowestReachable(minNumberOfMembers: 2) settings.cluster.downingStrategy = .none diff --git a/Samples/Sources/SampleCluster/main.swift b/Samples/Sources/SampleCluster/main.swift index df434bb79..ebcbf37e0 100644 --- a/Samples/Sources/SampleCluster/main.swift +++ b/Samples/Sources/SampleCluster/main.swift @@ -106,4 +106,4 @@ if system.cluster.node.port == 7337 { // <2> // end::cluster-sample-actors-discover-and-chat[] -system.park(atMost: .seconds(60)) +system.park(atMost: .seconds(6000)) diff --git a/Sources/DistributedActors/Cluster/Association.swift b/Sources/DistributedActors/Cluster/Association.swift index 98b4f5169..3cc16b75a 100644 --- a/Sources/DistributedActors/Cluster/Association.swift +++ b/Sources/DistributedActors/Cluster/Association.swift @@ -17,7 +17,7 @@ import Logging import NIO // ==== ---------------------------------------------------------------------------------------------------------------- -// MARK: Remote Association State Machine +// MARK: Association /// An `Association` represents a bi-directional agreement between two nodes that they are able to communicate with each other. /// @@ -32,98 +32,95 @@ import NIO /// /// A completed ("associated") `Association` can ONLY be obtained by successfully completing a `HandshakeStateMachine` dance, /// as only the handshake can ensure that the other side is also an actor node that is able and willing to communicate with us. -struct Association { +final class Association: CustomStringConvertible { + // TODO: Terrible lock which we want to get rid of; it means that every remote send has to content against all other sends about getting this ref + // and the only reason is really because the off chance case in which we have to make an Association earlier than we have the handshake completed (i.e. we send to a ref that is not yet associated) + let lock: Lock + + // TODO: This style of implementation queue -> channel swapping can only ever work with coarse locking and is just temporary + // We'd prefer to have a lock-less way to implement this and we can achieve it but it's a pain to implement so will be done in a separate step. + var state: State + + enum State { + case associating(queue: MPSCLinkedQueue) + case associated(channel: Channel) // TODO: ActorTransport.Node/Peer/Target ??? + case tombstone(ActorRef) + } - final class AssociationState: CustomStringConvertible { - // TODO: Terrible lock which we want to get rid of; it means that every remote send has to content against all other sends about getting this ref - // and the only reason is really because the off chance case in which we have to make an Association earlier than we have the handshake completed (i.e. we send to a ref that is not yet associated) - let lock: Lock + /// The address of this node, that was offered to the remote side for this association + /// This matters in case we have multiple "self" addresses; e.g. we bind to one address, but expose another because NAT + let selfNode: UniqueNode + var remoteNode: UniqueNode - // TODO: This style of implementation queue -> channel swapping can only ever work with coarse locking and is just temporary - // We'd prefer to have a lock-less way to implement this and we can achieve it but it's a pain to implement so will be done in a separate step. - var state: State + init(selfNode: UniqueNode, remoteNode: UniqueNode) { + self.selfNode = selfNode + self.remoteNode = remoteNode + self.lock = Lock() + self.state = .associating(queue: .init()) + } - enum State { - case associating(queue: MPSCLinkedQueue) - case associated(channel: Channel) // TODO: ActorTransport.Node/Peer/Target ??? - case tombstone(ActorRef) - } + /// Complete the association and drain any pending message sends onto the channel. + // TODO: This style can only ever work since we lock around the entirety of enqueueing messages and this setting; make it such that we don't need the lock eventually + func completeAssociation(handshake: HandshakeStateMachine.CompletedState, over channel: Channel) { + // TODO: assert that the channel is for the right remote node? - /// The address of this node, that was offered to the remote side for this association - /// This matters in case we have multiple "self" addresses; e.g. we bind to one address, but expose another because NAT - let selfNode: UniqueNode - var remoteNode: UniqueNode + self.lock.withLockVoid { + switch self.state { + case .associating(let sendQueue): + // 1) store associated channel + self.state = .associated(channel: channel) + + // 2) we need to flush all the queued up messages + // - yes, we need to flush while holding the lock... it's an annoyance in this lock based design + // but it ensures that once we've flushed, all other messages will be sent in the proper order "after" + // the previously enqueued ones; A lockless design would not be able to get rid of the queue AFAIR, + while let envelope = sendQueue.dequeue() { + _ = channel.writeAndFlush(envelope) + } - init(selfNode: UniqueNode, remoteNode: UniqueNode) { - self.selfNode = selfNode - self.remoteNode = remoteNode - self.lock = Lock() - self.state = .associating(queue: .init()) - } + case .associated(let existingAssociatedChannel): + fatalError("MUST NOT complete an association twice; Was \(existingAssociatedChannel) and tried to complete with \(channel) from \(handshake)") - /// Complete the association and drain any pending message sends onto the channel. - // TODO: This style can only ever work since we lock around the entirety of enqueueing messages and this setting; make it such that we don't need the lock eventually - func completeAssociation(handshake: HandshakeStateMachine.CompletedState, over channel: Channel) { - // TODO: assert that the channel is for the right remote node? - - self.lock.withLockVoid { - switch self.state { - case .associating(let sendQueue): - // 1) store associated channel - self.state = .associated(channel: channel) - - // 2) we need to flush all the queued up messages - // - yes, we need to flush while holding the lock... it's an annoyance in this lock based design - // but it ensures that once we've flushed, all other messages will be sent in the proper order "after" - // the previously enqueued ones; A lockless design would not be able to get rid of the queue AFAIR, - while let envelope = sendQueue.dequeue() { - _ = channel.writeAndFlush(envelope) - } - - case .associated(let existingAssociatedChannel): - fatalError("MUST NOT complete an association twice; Was \(existingAssociatedChannel) and tried to complete with \(channel) from \(handshake)") - - case .tombstone: - _ = channel.close() - return - } + case .tombstone: + _ = channel.close() + return } } + } - /// Terminate the association and store a tombstone in it. - /// - /// If any messages were still queued up in it, or if it was hosting a channel these get drained / closed, - /// before the tombstone is returned. - /// - /// After invoking this the association will never again be useful for sending messages. - func terminate(_ system: ActorSystem) -> Association.Tombstone { - self.lock.withLockVoid { - switch self.state { - case .associating(let sendQueue): - while let envelope = sendQueue.dequeue() { - system.deadLetters.tell(.init(envelope.underlyingMessage, recipient: envelope.recipient)) - } - // in case someone stored a reference to this association in a ref, we swap it into a dead letter sink - self.state = .tombstone(system.deadLetters) - case .associated(let channel): - _ = channel.close() - // in case someone stored a reference to this association in a ref, we swap it into a dead letter sink - self.state = .tombstone(system.deadLetters) - case .tombstone: - () // ok + /// Terminate the association and store a tombstone in it. + /// + /// If any messages were still queued up in it, or if it was hosting a channel these get drained / closed, + /// before the tombstone is returned. + /// + /// After invoking this the association will never again be useful for sending messages. + func terminate(_ system: ActorSystem) -> Association.Tombstone { + self.lock.withLockVoid { + switch self.state { + case .associating(let sendQueue): + while let envelope = sendQueue.dequeue() { + system.deadLetters.tell(.init(envelope.underlyingMessage, recipient: envelope.recipient)) } + // in case someone stored a reference to this association in a ref, we swap it into a dead letter sink + self.state = .tombstone(system.deadLetters) + case .associated(let channel): + _ = channel.close() + // in case someone stored a reference to this association in a ref, we swap it into a dead letter sink + self.state = .tombstone(system.deadLetters) + case .tombstone: + () // ok } - - return Association.Tombstone(self.remoteNode, settings: system.settings.cluster) } - var description: String { - "AssociatedState(\(self.state), selfNode: \(self.selfNode), remoteNode: \(self.remoteNode))" - } + return Association.Tombstone(self.remoteNode, settings: system.settings.cluster) + } + + var description: String { + "AssociatedState(\(self.state), selfNode: \(self.selfNode), remoteNode: \(self.remoteNode))" } } -extension Association.AssociationState { +extension Association { /// Concurrency: safe to invoke from any thread. func sendUserMessage(envelope: Envelope, recipient: ActorAddress, promise: EventLoopPromise? = nil) { let transportEnvelope = TransportEnvelope(envelope: envelope, recipient: recipient) @@ -152,7 +149,17 @@ extension Association.AssociationState { } } +// ==== ---------------------------------------------------------------------------------------------------------------- +// MARK: Association Tombstone + extension Association { + /// A tombstone is necessary to be kept for a period of time when an association is terminated, + /// as we may never be completely sure if the other node is truly terminated or just had network (or other issues) + /// for some time. In the case it'd try to associate and communicate with us again, we must be able to reject it + /// as we've terminated the association already, yet it may not have done so for its association to us. + /// + /// Tombstones are slightly lighter than a real association, and are kept for a maximum of `settings.cluster.associationTombstoneTTL` TODO: make this setting (!!!) + /// before being cleaned up. struct Tombstone: Hashable { let remoteNode: UniqueNode diff --git a/Sources/DistributedActors/Cluster/Cluster+Gossip.swift b/Sources/DistributedActors/Cluster/Cluster+Gossip.swift index edd043a5a..bfa091e8f 100644 --- a/Sources/DistributedActors/Cluster/Cluster+Gossip.swift +++ b/Sources/DistributedActors/Cluster/Cluster+Gossip.swift @@ -278,12 +278,12 @@ extension Cluster.Gossip { } } -extension Cluster.Gossip.SeenTable: CustomStringConvertible, CustomDebugStringConvertible { +extension Cluster.Gossip.SeenTable: CustomStringConvertible, CustomPrettyStringConvertible { public var description: String { "Cluster.Gossip.SeenTable(\(self.underlying))" } - var debugDescription: String { + public var prettyDescription: String { var s = "Cluster.Gossip.SeenTable(\n" let entryHeadingPadding = String(repeating: " ", count: 4) let entryPadding = String(repeating: " ", count: 4 * 2) diff --git a/Sources/DistributedActors/Cluster/ClusterSettings.swift b/Sources/DistributedActors/Cluster/ClusterSettings.swift index c292c57cb..4df7994e0 100644 --- a/Sources/DistributedActors/Cluster/ClusterSettings.swift +++ b/Sources/DistributedActors/Cluster/ClusterSettings.swift @@ -22,7 +22,7 @@ import NIOSSL public struct ClusterSettings { public enum Default { public static let systemName: String = "ActorSystem" - public static let bindHost: String = "localhost" + public static let bindHost: String = "127.0.0.1" public static let bindPort: Int = 7337 } @@ -157,7 +157,7 @@ public struct ClusterSettings { // MARK: Logging /// If enabled, logs membership changes (including the entire membership table from the perspective of the current node). - public var logMembershipChanges: Logger.Level? = .info + public var logMembershipChanges: Logger.Level? = .debug /// When enabled traces _all_ incoming and outgoing cluster (e.g. handshake) protocol communication (remote messages). /// All logs will be prefixed using `[tracelog:cluster]`, for easier grepping. diff --git a/Sources/DistributedActors/Cluster/ClusterShell.swift b/Sources/DistributedActors/Cluster/ClusterShell.swift index e30782b20..0405a236f 100644 --- a/Sources/DistributedActors/Cluster/ClusterShell.swift +++ b/Sources/DistributedActors/Cluster/ClusterShell.swift @@ -43,13 +43,13 @@ internal class ClusterShell { /// Used by remote actor refs to obtain associations /// - Protected by: `_associationsLock` - private var _activeAssociations: [UniqueNode: Association.AssociationState] + private var _activeAssociations: [UniqueNode: Association] /// Node tombstones are kept here in order to avoid attempting to associate if we get a reference to such node, /// which would normally trigger an `ensureAssociated`. /// - Protected by: `_associationsLock` private var _associationTombstones: [UniqueNode: Association.Tombstone] - internal func getExistingAssociation(with node: Node) -> Association.AssociationState? { + internal func getExistingAssociation(with node: Node) -> Association? { self._associationsLock.withLock { // TODO: a bit terrible; perhaps we should key be Node and then confirm by UniqueNode? // this used to be separated in the State keeping them by Node and here we kept by unique though that caused other challenges @@ -68,7 +68,7 @@ internal class ClusterShell { } else if let existing = self._activeAssociations[node] { return .association(existing) } else { - let association = Association.AssociationState(selfNode: self.selfNode, remoteNode: node) + let association = Association(selfNode: self.selfNode, remoteNode: node) self._activeAssociations[node] = association /// We're trying to send to `node` yet it has no association (not even in progress), @@ -82,7 +82,7 @@ internal class ClusterShell { enum StoredAssociationState { /// An existing (ready or being associated association) which can be used to send (or buffer buffer until associated/terminated) - case association(Association.AssociationState) + case association(Association) /// The association with the node existed, but is now a tombstone and no more messages shall be send to it. case tombstone(Association.Tombstone) } @@ -92,7 +92,7 @@ internal class ClusterShell { private func storeCompleteAssociation(_ associated: ClusterShellState.AssociatedDirective) { self._associationsLock.withLockVoid { let association = self._activeAssociations[associated.handshake.remoteNode] ?? - Association.AssociationState(selfNode: associated.handshake.localNode, remoteNode: associated.handshake.remoteNode) + Association(selfNode: associated.handshake.localNode, remoteNode: associated.handshake.remoteNode) association.completeAssociation(handshake: associated.handshake, over: associated.channel) @@ -109,7 +109,7 @@ internal class ClusterShell { internal func terminateAssociation(_ system: ActorSystem, state: inout ClusterShellState, _ remoteNode: UniqueNode) { traceLog_Remote(system.cluster.node, "Terminate association with [\(remoteNode)]") - let removedAssociationOption: Association.AssociationState? = self._associationsLock.withLock { + let removedAssociationOption: Association? = self._associationsLock.withLock { // tombstone the association in the shell immediately. // No more message sends to the system will be possible. traceLog_Remote(system.cluster.node, "Finish terminate association [\(remoteNode)]: Stored tombstone") @@ -147,7 +147,7 @@ internal class ClusterShell { /// This is a best-effort message; as we may be downing it because we cannot communicate with it after all, in such situation (and many others) /// the other node would never receive this direct kill/down eager "gossip." We hope it will either receive the down via some means, or determine /// by itself that it should down itself. - internal static func shootTheOtherNodeAndCloseConnection(system: ActorSystem, targetNodeAssociation: Association.AssociationState) { + internal static func shootTheOtherNodeAndCloseConnection(system: ActorSystem, targetNodeAssociation: Association) { let log = system.log let remoteNode = targetNodeAssociation.remoteNode traceLog_Remote(system.cluster.node, "Finish terminate association [\(remoteNode)]: Shooting the other node a direct .gossip to down itself") @@ -178,9 +178,9 @@ internal class ClusterShell { /// For testing only. /// Safe to concurrently access by privileged internals. - internal var _testingOnly_associations: [Association.AssociationState] { + internal var _testingOnly_associations: [Association] { self._associationsLock.withLock { - [Association.AssociationState](self._activeAssociations.values) + [Association](self._activeAssociations.values) } } diff --git a/Sources/DistributedActors/Cluster/ClusterShellState.swift b/Sources/DistributedActors/Cluster/ClusterShellState.swift index ba7c08058..69f2bcfad 100644 --- a/Sources/DistributedActors/Cluster/ClusterShellState.swift +++ b/Sources/DistributedActors/Cluster/ClusterShellState.swift @@ -343,7 +343,7 @@ extension ClusterShellState { struct AssociatedDirective { let membershipChange: Cluster.MembershipChange - // let association: Association.AssociationState + // let association: Association let handshake: HandshakeStateMachine.CompletedState let channel: Channel } diff --git a/Sources/DistributedActors/Cluster/SWIM/SWIMInstance.swift b/Sources/DistributedActors/Cluster/SWIM/SWIMInstance.swift index cac2ff9c3..7935109fc 100644 --- a/Sources/DistributedActors/Cluster/SWIM/SWIMInstance.swift +++ b/Sources/DistributedActors/Cluster/SWIM/SWIMInstance.swift @@ -66,11 +66,11 @@ final class SWIMInstance { var localHealthMultiplier = 0 var dynamicLHMProtocolInterval: TimeAmount { - TimeAmount.nanoseconds(self.settings.failureDetector.probeInterval.nanoseconds * Int64(1 + self.localHealthMultiplier)) + TimeAmount.nanoseconds(self.settings.probeInterval.nanoseconds * Int64(1 + self.localHealthMultiplier)) } var dynamicLHMPingTimeout: TimeAmount { - TimeAmount.nanoseconds(self.settings.failureDetector.pingTimeout.nanoseconds * Int64(1 + self.localHealthMultiplier)) + TimeAmount.nanoseconds(self.settings.pingTimeout.nanoseconds * Int64(1 + self.localHealthMultiplier)) } /// The incarnation number is used to get a sense of ordering of events, so if an `.alive` or `.suspect` @@ -198,7 +198,7 @@ final class SWIMInstance { return self.membersToPing[self.membersToPingIndex].ref } - /// Selects `settings.failureDetector.indirectProbeCount` members to send a `ping-req` to. + /// Selects `settings.indirectProbeCount` members to send a `ping-req` to. func membersToPingRequest(target: ActorRef) -> ArraySlice { func notTarget(_ ref: ActorRef) -> Bool { ref.address != target.address @@ -211,7 +211,7 @@ final class SWIMInstance { .filter { notTarget($0.ref) && notMyself($0.ref) && isReachable($0.status) } .shuffled() - return candidates.prefix(self.settings.failureDetector.indirectProbeCount) + return candidates.prefix(self.settings.indirectProbeCount) } func notMyself(_ member: SWIM.Member) -> Bool { diff --git a/Sources/DistributedActors/Cluster/SWIM/SWIMSettings.swift b/Sources/DistributedActors/Cluster/SWIM/SWIMSettings.swift index 46cc4bde3..06bbb9979 100644 --- a/Sources/DistributedActors/Cluster/SWIM/SWIMSettings.swift +++ b/Sources/DistributedActors/Cluster/SWIM/SWIMSettings.swift @@ -18,6 +18,10 @@ import Logging // ==== ---------------------------------------------------------------------------------------------------------------- // MARK: SWIM Settings +extension SWIM { + public typealias Settings = SWIMSettings +} + public struct SWIMSettings { public static var `default`: SWIMSettings { .init() @@ -33,9 +37,38 @@ public struct SWIMSettings { public var gossip: SWIMGossipSettings = .default - public var lifeguard: SWIMLifeGuardSettings = .default + /// Number of indirect probes that will be issued once a direct ping probe has failed to reply in time with an ack. + /// + /// In case of small clusters where nr. of neighbors is smaller than this value, the most neighbors available will + /// be asked to issue an indirect probe. E.g. a 3 node cluster, configured with `indirectChecks = 3` has only `1` + /// remaining node it can ask for an indirect probe (since 1 node is ourselves, and 1 node is the potentially suspect node itself). + public var indirectProbeCount: Int = 3 { + willSet { + precondition(newValue >= 0, "`indirectChecks` MUST be >= 0. It is recommended to have it be no lower than 3.") + } + } + + /// Interval at which gossip messages should be issued. + /// This property sets only a base value of probe interval, which will later be multiplied by `localHealthMultiplier`. + /// - SeeAlso: `maxLocalHealthMultiplier` + /// Every `interval` a `fanout` number of gossip messages will be sent. // TODO which fanout? + public var probeInterval: TimeAmount = .seconds(1) + + /// Time amount after which a sent ping without ack response is considered timed-out. + /// This drives how a node becomes a suspect, by missing such ping/ack rounds. + /// + /// This property sets only a base timeout value, which is later multiplied by `localHealthMultiplier` + /// Note that after an initial ping/ack timeout, secondary indirect probes are issued, + /// and only after exceeding `suspicionTimeoutPeriodsMax` shall the node be declared as `.unreachable`, + /// which results in an `Cluster.MemberReachabilityChange` `Cluster.Event` which downing strategies may act upon. + /// + /// - SeeAlso: `lifeguard.maxLocalHealthMultiplier` + public var pingTimeout: TimeAmount = .milliseconds(300) - public var failureDetector: SWIMFailureDetectorSettings = .default + /// Settings of the Lifeguard extensions to the SWIM protocol. + /// + /// - SeeAlso: `SWIMLifeguardSettings` for in depth documentation about it. + public var lifeguard: SWIMLifeguardSettings = .default /// Optional "SWIM instance name" to be included in log statements, /// useful when multiple instances of SWIM are run on the same node (e.g. for debugging). @@ -52,10 +85,6 @@ public struct SWIMSettings { #endif } -extension SWIM { - public typealias Settings = SWIMSettings -} - // ==== ---------------------------------------------------------------------------------------------------------------- // MARK: SWIM Gossip Settings @@ -72,73 +101,44 @@ public struct SWIMGossipSettings { } // ==== ---------------------------------------------------------------------------------------------------------------- -// MARK: SWIM FailureDetector Settings - -public struct SWIMFailureDetectorSettings { - public static var `default`: SWIMFailureDetectorSettings { - .init() - } - - /// Number of indirect probes that will be issued once a direct ping probe has failed to reply in time with an ack. - /// - /// In case of small clusters where nr. of neighbors is smaller than this value, the most neighbors available will - /// be asked to issue an indirect probe. E.g. a 3 node cluster, configured with `indirectChecks = 3` has only `1` - /// remaining node it can ask for an indirect probe (since 1 node is ourselves, and 1 node is the potentially suspect node itself). - public var indirectProbeCount: Int = 3 { - willSet { - precondition(newValue >= 0, "`indirectChecks` MUST NOT be < 0. It is recommended to have it be no lower than 3.") - } - } - - /// Interval at which gossip messages should be issued. - /// This property sets only a base value of probe interval, which will later be multiplied by `localHealthMultiplier`. - /// - SeeAlso: `maxLocalHealthMultiplier` - /// Every `interval` a `fanout` number of gossip messages will be sent. // TODO which fanout? - public var probeInterval: TimeAmount = .seconds(1) - - /// Time amount after which a sent ping without ack response is considered timed-out. - /// This drives how a node becomes a suspect, by missing such ping/ack rounds. - /// - /// This property sets only a base timeout value, which is later multiplied by `localHealthMultiplier` - /// - SeeAlso: `maxLocalHealthMultiplier` - /// Note that after an initial ping/ack timeout, secondary indirect probes are issued, - /// and only after exceeding `suspicionTimeoutPeriodsMax` shall the node be declared as `.unreachable`, - /// which results in an `Cluster.MemberReachabilityChange` `Cluster.Event` which downing strategies may act upon. - public var pingTimeout: TimeAmount = .milliseconds(300) -} - -// ==== ---------------------------------------------------------------------------------------------------------------- -// MARK: SWIM LifeGuard extensions Settings +// MARK: SWIM Lifeguard extensions Settings /// Lifeguard is a set of extensions to SWIM that helps reducing false positive failure detections -/// Extensions description: https://arxiv.org/pdf/1707.00788.pdf -public struct SWIMLifeGuardSettings { - public static var `default`: SWIMLifeGuardSettings { +/// +/// - SeeAlso: [Lifeguard: Local Health Awareness for More Accurate Failure Detection](https://arxiv.org/pdf/1707.00788.pdf) +public struct SWIMLifeguardSettings { + public static var `default`: SWIMLifeguardSettings { .init() } - /// This is not a part of public API. SWIM is using time to schedule pings/calculate timeouts. - /// When designing tests one may want to simulate scenarious when events are coming in particular order. - /// Doing this will require some control over SWIM's notion of time. - /// This propery allows to override the `.now` function. - var timeSourceNanos: () -> Int64 = { () -> Int64 in Int64(DispatchTime.now().uptimeNanoseconds) } - /// Local health multiplier is a part of Lifeguard extensions to SWIM. /// It will increase local probe interval and probe timeout if the instance is not processing messages in timely manner. - /// This property will define the upper limit to local health multiplier. The lower bound is always 0. + /// This property will define the upper limit to local health multiplier. + /// + /// Must be greater than 0. To effectively disable the LHM extension you may set this to `1`. + /// + /// - SeeAlso: [Lifeguard IV.A. Local Health Multiplier (LHM)](https://arxiv.org/pdf/1707.00788.pdf) public var maxLocalHealthMultiplier: Int = 8 { willSet { - precondition(newValue >= 0, "Local health multiplier CAN NOT be negative") + precondition(newValue >= 0, "Local health multiplier MUST BE >= 0") } } /// Suspicion timeouts are specified as number of probe intervals. - /// E.g. a `suspicionTimeoutMax = .seconds(10)` means that a suspicious node - /// will be escalated as `.unreachable` at most after approximately 10 seconds. Suspicion timeout will decay logarithmically to `suspicionTimeoutMin` + /// + /// E.g. a `suspicionTimeoutMax = .seconds(10)` means that a suspicious node will be escalated as `.unreachable` at most after approximately 10 seconds. Suspicion timeout will decay logarithmically to `suspicionTimeoutMin` /// with additional suspicions arriving. When no additional suspicions present, suspicion timeout will equal `suspicionTimeoutMax` /// - /// Once it is confirmed dead by the high-level membership (e.g. immediately, or after an additional grace period, or vote), it will be marked `.dead` in swim, - /// and `.down` in the high-level membership. + /// ### Distributed Actors modification: + /// In Distributed Actors, an extra state of "unreachable" is introduced, which is signalled to a high-level membership implementation, + /// which may then confirm it, then leading the SWIM membership to mark the given member as `.dead`. Unlike the original SWIM/Lifeguard + /// implementations which proceed to `.dead` automatically. This separation allows running with SWIM failure detection in an "informational" + /// mode. + /// + /// Once it is confirmed dead by the high-level membership (e.g. immediately, or after an additional grace period, or vote), + /// it will be marked `.dead` in SWIM, and `.down` in the high-level membership. + /// + /// - SeeAlso: [Lifeguard IV.B. Local Health Aware Suspicion (LHA-Suspicion)](https://arxiv.org/pdf/1707.00788.pdf) public var suspicionTimeoutMax: TimeAmount = .seconds(10) { willSet { precondition(newValue >= self.suspicionTimeoutMin, "`suspicionTimeoutMax` MUST BE >= `suspicionTimeoutMin`") @@ -146,24 +146,39 @@ public struct SWIMLifeGuardSettings { } /// Suspicion timeouts are specified as number of probe intervals. - /// E.g. a `suspicionTimeoutMin = .seconds(3)` means that a suspicious node - /// will be escalated as `.unreachable` at least after approximately 3 seconds. Suspicion timeout will decay logarithmically from `suspicionTimeoutMax` - /// with additional suspicions arriving. When number of suspicions reach `maxIndependentSuspicions`, suspicion timeout will equal `suspicionTimeoutMin` /// - /// Once it is confirmed dead by the high-level membership (e.g. immediately, or after an additional grace period, or vote), it will be marked `.dead` in swim, - /// and `.down` in the high-level membership. + /// E.g. a `suspicionTimeoutMin = .seconds(3)` means that a suspicious node will be escalated as `.unreachable` at least after approximately 3 seconds. + /// Suspicion timeout will decay logarithmically from `suspicionTimeoutMax` / with additional suspicions arriving. + /// When number of suspicions reach `maxIndependentSuspicions`, suspicion timeout will equal `suspicionTimeoutMin` + /// + /// ### Distributed Actors modification: + /// In Distributed Actors, an extra state of "unreachable" is introduced, which is signalled to a high-level membership implementation, + /// which may then confirm it, then leading the SWIM membership to mark the given member as `.dead`. Unlike the original SWIM/Lifeguard + /// implementations which proceed to `.dead` automatically. This separation allows running with SWIM failure detection in an "informational" + /// mode. + /// + /// Once it is confirmed dead by the high-level membership (e.g. immediately, or after an additional grace period, or vote), + /// it will be marked `.dead` in swim, and `.down` in the high-level membership. + /// + /// - SeeAlso: [Lifeguard IV.B. Local Health Aware Suspicion (LHA-Suspicion)](https://arxiv.org/pdf/1707.00788.pdf) public var suspicionTimeoutMin: TimeAmount = .seconds(3) { willSet { precondition(newValue <= self.suspicionTimeoutMax, "`suspicionTimeoutMin` MUST BE <= `suspicionTimeoutMax`") } } - /// A Lifegurad suspicion extension to SWIM protocol. /// A number of independent suspicions required for a suspicion timeout to fully decay to a minimal value. - /// When set to 1 will effectively disable LHA-suspicion + /// When set to 1 will effectively disable LHA-suspicion. public var maxIndependentSuspicions = 4 { willSet { - precondition(newValue >= 0, "`maxIndependentSuspicions` MUST BE > 0") + precondition(newValue > 0, "`settings.cluster.swim.maxIndependentSuspicions` MUST BE > 0") } } + + /// This is not a part of public API. SWIM is using time to schedule pings/calculate timeouts. + /// When designing tests one may want to simulate scenarios when events are coming in particular order. + /// Doing this will require some control over SWIM's notion of time. + /// + /// This property allows to override the `.now` function. // TODO: replace with always using the `system.now()` clock. + var timeSourceNanos: () -> Int64 = { () -> Int64 in Int64(DispatchTime.now().uptimeNanoseconds) } } diff --git a/Sources/DistributedActors/Cluster/SWIM/SWIMShell.swift b/Sources/DistributedActors/Cluster/SWIM/SWIMShell.swift index b3cf63138..072e214b5 100644 --- a/Sources/DistributedActors/Cluster/SWIM/SWIMShell.swift +++ b/Sources/DistributedActors/Cluster/SWIM/SWIMShell.swift @@ -39,7 +39,7 @@ internal struct SWIMShell { // TODO: install an .cluster.down(my node) with context.defer in case we crash? Or crash system when this crashes: issue #926 - let probeInterval = settings.failureDetector.probeInterval + let probeInterval = settings.probeInterval context.timers.startSingle(key: SWIM.Shell.periodicPingKey, message: .local(.pingRandomMember), delay: probeInterval) let shell = SWIMShell(SWIMInstance(settings, myShellMyself: context.myself, myNode: context.system.cluster.node), clusterRef: clusterRef) @@ -235,7 +235,7 @@ internal struct SWIMShell { switch result { case .failure(let err): if let timeoutError = err as? TimeoutError { - context.log.warning( + context.log.debug( """ Did not receive ack from \(reflecting: pingedMember.address) within [\(timeoutError.timeout.prettyDescription)]. \ Sending ping requests to other members. diff --git a/Sources/DistributedActors/Cluster/SystemMessages+Redelivery.swift b/Sources/DistributedActors/Cluster/SystemMessages+Redelivery.swift index b2fcca657..c074f1b8a 100644 --- a/Sources/DistributedActors/Cluster/SystemMessages+Redelivery.swift +++ b/Sources/DistributedActors/Cluster/SystemMessages+Redelivery.swift @@ -260,7 +260,7 @@ internal final class OutboundSystemMessageRedelivery { // } } - func onReconnected(newAssociationID: Association.AssociationState) { + func onReconnected(newAssociationID: Association) { // TODO: redeliver everything } diff --git a/Sources/DistributedActors/Receptionist.swift b/Sources/DistributedActors/Receptionist.swift index 78bbc8afa..72aaacd40 100644 --- a/Sources/DistributedActors/Receptionist.swift +++ b/Sources/DistributedActors/Receptionist.swift @@ -305,8 +305,6 @@ public enum Receptionist { // 1) we remove any registrations that it hosted let regs: Set = self._registrations.removeValue(forKey: key) ?? [] let prunedRegs = regs.filter { $0.address.node != node } - pprint("regs = \(regs)") - pprint("prunedRegs = \(prunedRegs)") if !prunedRegs.isEmpty { self._registrations[key] = prunedRegs } diff --git a/Sources/DistributedActors/CustomStringInterpolations.swift b/Sources/DistributedActors/String+Extensions.swift similarity index 84% rename from Sources/DistributedActors/CustomStringInterpolations.swift rename to Sources/DistributedActors/String+Extensions.swift index 1b283267f..e2ee5754a 100644 --- a/Sources/DistributedActors/CustomStringInterpolations.swift +++ b/Sources/DistributedActors/String+Extensions.swift @@ -12,6 +12,16 @@ // //===----------------------------------------------------------------------===// +// ==== ---------------------------------------------------------------------------------------------------------------- +// MARK: Pretty String Descriptions + +/// Marks a type that can be "pretty" printed, meaning often multi-line well formatted/aligned. +protocol CustomPrettyStringConvertible { + /// Pretty representation of the type, intended for inspection in command line and "visual" inspection. + /// Not to be used in log statements or otherwise persisted formats. + var prettyDescription: String { get } +} + // ==== ---------------------------------------------------------------------------------------------------------------- // MARK: String Interpolation: _:leftPad: diff --git a/Sources/DistributedActors/Time.swift b/Sources/DistributedActors/Time.swift index 1b48f3c05..0ff434f07 100644 --- a/Sources/DistributedActors/Time.swift +++ b/Sources/DistributedActors/Time.swift @@ -124,7 +124,7 @@ extension TimeAmount: Comparable { } /// "Pretty" time amount rendering, useful for human readable durations in tests -extension TimeAmount: CustomStringConvertible { +extension TimeAmount: CustomStringConvertible, CustomPrettyStringConvertible { public var description: String { "TimeAmount(\(self.prettyDescription), nanoseconds: \(self.nanoseconds))" } diff --git a/Tests/DistributedActorsTests/Actorable/ActorContextReceptionistTests.swift b/Tests/DistributedActorsTests/Actorable/ActorContextReceptionistTests.swift index 6a0f06392..9ab885f1d 100644 --- a/Tests/DistributedActorsTests/Actorable/ActorContextReceptionistTests.swift +++ b/Tests/DistributedActorsTests/Actorable/ActorContextReceptionistTests.swift @@ -85,7 +85,7 @@ final class ActorContextReceptionTests: ActorSystemTestBase { func test_autoUpdatedListing_shouldQuicklyUpdateFromThousandsOfUpdates() throws { let p = self.testKit.spawnTestProbe(expecting: Reception.Listing.self) - let n = 3000 + let n = 2000 _ = try! self.system.spawn("owner") { OwnerOfThings( diff --git a/Tests/DistributedActorsTests/CRDT/Protobuf/CRDT+SerializationTests.swift b/Tests/DistributedActorsTests/CRDT/Protobuf/CRDT+SerializationTests.swift index e210a1ca2..2d7b197f3 100644 --- a/Tests/DistributedActorsTests/CRDT/Protobuf/CRDT+SerializationTests.swift +++ b/Tests/DistributedActorsTests/CRDT/Protobuf/CRDT+SerializationTests.swift @@ -68,11 +68,11 @@ final class CRDTSerializationTests: ActorSystemTestBase { let deserialized = try system.serialization.deserialize(as: CRDT.VersionContext.self, from: serialized) deserialized.vv.state.count.shouldEqual(2) // replicas alpha and beta - "\(deserialized.vv)".shouldContain("actor:sact://CRDTSerializationTests@localhost:9001/user/alpha: 1") - "\(deserialized.vv)".shouldContain("actor:sact://CRDTSerializationTests@localhost:9001/user/beta: 3") + "\(deserialized.vv)".shouldContain("actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/alpha: 1") + "\(deserialized.vv)".shouldContain("actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/beta: 3") deserialized.gaps.count.shouldEqual(1) - "\(deserialized.gaps)".shouldContain("Dot(actor:sact://CRDTSerializationTests@localhost:9001/user/alpha,4)") + "\(deserialized.gaps)".shouldContain("Dot(actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/alpha,4)") } } @@ -109,22 +109,22 @@ final class CRDTSerializationTests: ActorSystemTestBase { let serialized = try system.serialization.serialize(versionedContainer) let deserialized = try system.serialization.deserialize(as: CRDT.VersionedContainer.self, from: serialized) - "\(deserialized.replicaID)".shouldContain("actor:sact://CRDTSerializationTests@localhost:9001/user/alpha") - "\(deserialized.versionContext.vv)".shouldContain("actor:sact://CRDTSerializationTests@localhost:9001/user/alpha: 3") // adding "bye" bumps version to 3 - "\(deserialized.versionContext.vv)".shouldContain("actor:sact://CRDTSerializationTests@localhost:9001/user/beta: 1") - "\(deserialized.versionContext.gaps)".shouldContain("Dot(actor:sact://CRDTSerializationTests@localhost:9001/user/beta,3)") + "\(deserialized.replicaID)".shouldContain("actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/alpha") + "\(deserialized.versionContext.vv)".shouldContain("actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/alpha: 3") // adding "bye" bumps version to 3 + "\(deserialized.versionContext.vv)".shouldContain("actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/beta: 1") + "\(deserialized.versionContext.gaps)".shouldContain("Dot(actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/beta,3)") deserialized.elementByBirthDot.count.shouldEqual(3) - "\(deserialized.elementByBirthDot)".shouldContain("Dot(actor:sact://CRDTSerializationTests@localhost:9001/user/alpha,1): \"hello\"") - "\(deserialized.elementByBirthDot)".shouldContain("Dot(actor:sact://CRDTSerializationTests@localhost:9001/user/beta,3): \"world\"") - "\(deserialized.elementByBirthDot)".shouldContain("Dot(actor:sact://CRDTSerializationTests@localhost:9001/user/alpha,3): \"bye\"") + "\(deserialized.elementByBirthDot)".shouldContain("Dot(actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/alpha,1): \"hello\"") + "\(deserialized.elementByBirthDot)".shouldContain("Dot(actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/beta,3): \"world\"") + "\(deserialized.elementByBirthDot)".shouldContain("Dot(actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/alpha,3): \"bye\"") deserialized.delta.shouldNotBeNil() // The birth dot for "bye" is added to gaps since delta's versionContext started out empty // and therefore not contiguous deserialized.delta!.versionContext.vv.isEmpty.shouldBeTrue() - "\(deserialized.delta!.versionContext.gaps)".shouldContain("Dot(actor:sact://CRDTSerializationTests@localhost:9001/user/alpha,3)") + "\(deserialized.delta!.versionContext.gaps)".shouldContain("Dot(actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/alpha,3)") deserialized.delta!.elementByBirthDot.count.shouldEqual(1) - "\(deserialized.delta!.elementByBirthDot)".shouldContain("Dot(actor:sact://CRDTSerializationTests@localhost:9001/user/alpha,3): \"bye\"") + "\(deserialized.delta!.elementByBirthDot)".shouldContain("Dot(actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/alpha,3): \"bye\"") } } @@ -135,7 +135,7 @@ final class CRDTSerializationTests: ActorSystemTestBase { let serialized = try system.serialization.serialize(versionedContainer) let deserialized = try system.serialization.deserialize(as: CRDT.VersionedContainer.self, from: serialized) - "\(deserialized.replicaID)".shouldContain("actor:sact://CRDTSerializationTests@localhost:9001/user/alpha") + "\(deserialized.replicaID)".shouldContain("actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/alpha") deserialized.versionContext.vv.isEmpty.shouldBeTrue() deserialized.versionContext.gaps.isEmpty.shouldBeTrue() deserialized.elementByBirthDot.isEmpty.shouldBeTrue() @@ -155,8 +155,8 @@ final class CRDTSerializationTests: ActorSystemTestBase { let deserialized = try system.serialization.deserialize(as: CRDT.GCounter.self, from: serialized) g1.value.shouldEqual(deserialized.value) - "\(deserialized.replicaID)".shouldContain("actor:sact://CRDTSerializationTests@localhost:9001/user/alpha") - "\(deserialized.state)".shouldContain("[actor:sact://CRDTSerializationTests@localhost:9001/user/alpha: 2]") + "\(deserialized.replicaID)".shouldContain("actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/alpha") + "\(deserialized.state)".shouldContain("[actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/alpha: 2]") } } @@ -168,7 +168,7 @@ final class CRDTSerializationTests: ActorSystemTestBase { let serialized = try system.serialization.serialize(g1.delta!) // !-safe, must have a delta, we just changed it let deserialized = try system.serialization.deserialize(as: CRDT.GCounter.Delta.self, from: serialized) - "\(deserialized.state)".shouldContain("[actor:sact://CRDTSerializationTests@localhost:9001/user/alpha: 13]") + "\(deserialized.state)".shouldContain("[actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/alpha: 13]") } } @@ -186,9 +186,9 @@ final class CRDTSerializationTests: ActorSystemTestBase { let serialized = try system.serialization.serialize(set) let deserialized = try system.serialization.deserialize(as: CRDT.ORSet.self, from: serialized) - "\(deserialized.replicaID)".shouldContain("actor:sact://CRDTSerializationTests@localhost:9001/user/alpha") + "\(deserialized.replicaID)".shouldContain("actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/alpha") deserialized.elements.shouldEqual(set.elements) - "\(deserialized.state.versionContext.vv)".shouldContain("[actor:sact://CRDTSerializationTests@localhost:9001/user/alpha: 2]") + "\(deserialized.state.versionContext.vv)".shouldContain("[actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/alpha: 2]") deserialized.state.versionContext.gaps.isEmpty.shouldBeTrue() // changes are contiguous so no gaps deserialized.state.elementByBirthDot.count.shouldEqual(2) "\(deserialized.state.elementByBirthDot)".shouldContain("/user/alpha,1): \"hello\"") @@ -210,7 +210,7 @@ final class CRDTSerializationTests: ActorSystemTestBase { let deserialized = try system.serialization.deserialize(as: CRDT.ORSet.Delta.self, from: serialized) // delta contains the same elements as set - "\(deserialized.versionContext.vv)".shouldContain("[actor:sact://CRDTSerializationTests@localhost:9001/user/alpha: 2]") + "\(deserialized.versionContext.vv)".shouldContain("[actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/alpha: 2]") deserialized.versionContext.gaps.isEmpty.shouldBeTrue() // changes are contiguous so no gaps deserialized.elementByBirthDot.count.shouldEqual(2) "\(deserialized.elementByBirthDot)".shouldContain("/user/alpha,1): \"hello\"") diff --git a/Tests/DistributedActorsTests/CRDT/Protobuf/CRDTEnvelope+SerializationTests.swift b/Tests/DistributedActorsTests/CRDT/Protobuf/CRDTEnvelope+SerializationTests.swift index e8c17245f..cd2ffc9e2 100644 --- a/Tests/DistributedActorsTests/CRDT/Protobuf/CRDTEnvelope+SerializationTests.swift +++ b/Tests/DistributedActorsTests/CRDT/Protobuf/CRDTEnvelope+SerializationTests.swift @@ -31,16 +31,13 @@ final class CRDTEnvelopeSerializationTests: ActorSystemTestBase { let serialized = try system.serialization.serialize(envelope) let deserialized = try system.serialization.deserialize(as: CRDT.Envelope.self, from: serialized) -// guard case let data = deserialized.data else { -// throw self.testKit.fail("CRDT.Envelope._boxed should be .DeltaCRDT for DeltaCRDTBox") -// } guard let gg1 = deserialized.data as? CRDT.GCounter else { throw self.testKit.fail("DeltaCRDTBox.underlying should be GCounter") } gg1.value.shouldEqual(g1.value) gg1.delta.shouldNotBeNil() - "\(gg1.delta!.state)".shouldContain("[actor:sact://CRDTEnvelopeSerializationTests@localhost:9001/user/alpha: 2]") + "\(gg1.delta!.state)".shouldContain("[actor:sact://CRDTEnvelopeSerializationTests@127.0.0.1:9001/user/alpha: 2]") } } @@ -65,7 +62,7 @@ final class CRDTEnvelopeSerializationTests: ActorSystemTestBase { // } // // dg1Delta.state.count.shouldEqual(1) -// "\(dg1Delta.state)".shouldContain("[actor:sact://CRDTEnvelopeSerializationTests@localhost:9001/user/alpha: 2]") +// "\(dg1Delta.state)".shouldContain("[actor:sact://CRDTEnvelopeSerializationTests@127.0.0.1:9001/user/alpha: 2]") // } // } } diff --git a/Tests/DistributedActorsTests/CRDT/Protobuf/CRDTReplication+SerializationTests.swift b/Tests/DistributedActorsTests/CRDT/Protobuf/CRDTReplication+SerializationTests.swift index bcf2098a3..27763ad0a 100644 --- a/Tests/DistributedActorsTests/CRDT/Protobuf/CRDTReplication+SerializationTests.swift +++ b/Tests/DistributedActorsTests/CRDT/Protobuf/CRDTReplication+SerializationTests.swift @@ -112,7 +112,7 @@ final class CRDTReplicationSerializationTests: ActorSystemTestBase { guard let ddg1 = deserializedDelta as? CRDT.GCounterDelta else { throw self.testKit.fail("Should be a GCounter") } - "\(ddg1.state)".shouldContain("[actor:sact://CRDTReplicationSerializationTests@localhost:9001/user/alpha: 5]") + "\(ddg1.state)".shouldContain("[actor:sact://CRDTReplicationSerializationTests@127.0.0.1:9001/user/alpha: 5]") } } diff --git a/Tests/DistributedActorsTests/Clocks/Protobuf/VersionVector+SerializationTests.swift b/Tests/DistributedActorsTests/Clocks/Protobuf/VersionVector+SerializationTests.swift index 9de0a9acc..ca9671740 100644 --- a/Tests/DistributedActorsTests/Clocks/Protobuf/VersionVector+SerializationTests.swift +++ b/Tests/DistributedActorsTests/Clocks/Protobuf/VersionVector+SerializationTests.swift @@ -36,8 +36,8 @@ final class VersionVectorSerializationTests: ActorSystemTestBase { let deserialized = try system.serialization.deserialize(as: VersionVector.self, from: serialized) deserialized.state.count.shouldEqual(2) // replicas A and B - "\(deserialized)".shouldContain("actor:sact://VersionVectorSerializationTests@localhost:9001/user/A: 2") - "\(deserialized)".shouldContain("actor:sact://VersionVectorSerializationTests@localhost:9001/user/B: 5") + "\(deserialized)".shouldContain("actor:sact://VersionVectorSerializationTests@127.0.0.1:9001/user/A: 2") + "\(deserialized)".shouldContain("actor:sact://VersionVectorSerializationTests@127.0.0.1:9001/user/B: 5") } } @@ -62,7 +62,7 @@ final class VersionVectorSerializationTests: ActorSystemTestBase { let serialized = try system.serialization.serialize(dot) let deserialized = try system.serialization.deserialize(as: VersionDot.self, from: serialized) - "\(deserialized)".shouldContain("actor:sact://VersionVectorSerializationTests@localhost:9001/user/A") + "\(deserialized)".shouldContain("actor:sact://VersionVectorSerializationTests@127.0.0.1:9001/user/A") deserialized.version.shouldEqual(2) } } diff --git a/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift b/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift index 461f46331..631126248 100644 --- a/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift +++ b/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift @@ -174,7 +174,7 @@ final class ClusterAssociationTests: ClusteredNodesTestBase { let remotePort = local.cluster.node.node.port + 10 // remote is NOT started, but we already ask local to handshake with the remote one (which will fail, though the node should keep trying) - let remoteNode = Node(systemName: "remote", host: "localhost", port: remotePort) + let remoteNode = Node(systemName: "remote", host: "127.0.0.1", port: remotePort) local.cluster.join(node: remoteNode) sleep(1) // we give it some time to keep failing to connect, so the second node is not yet started diff --git a/Tests/DistributedActorsTests/Cluster/ClusterLeaderActionsClusteredTests.swift b/Tests/DistributedActorsTests/Cluster/ClusterLeaderActionsClusteredTests.swift index 7e07a3839..3394867a1 100644 --- a/Tests/DistributedActorsTests/Cluster/ClusterLeaderActionsClusteredTests.swift +++ b/Tests/DistributedActorsTests/Cluster/ClusterLeaderActionsClusteredTests.swift @@ -19,6 +19,12 @@ import NIOSSL import XCTest final class ClusterLeaderActionsClusteredTests: ClusteredNodesTestBase { + override func configureLogCapture(settings: inout LogCapture.Settings) { + settings.filterActorPaths = [ + "/system/cluster/swim", + ] + } + // ==== ------------------------------------------------------------------------------------------------------------ // MARK: leader decision: .joining -> .up @@ -256,6 +262,8 @@ final class ClusterLeaderActionsClusteredTests: ClusteredNodesTestBase { func test_ensureDownAndRemovalSpreadsToAllMembers() throws { try shouldNotThrow { let first = self.setUpNode("first") { settings in + settings.cluster.swim.probeInterval = .milliseconds(300) + settings.cluster.swim.pingTimeout = .milliseconds(100) settings.cluster.autoLeaderElection = .lowestReachable(minNumberOfMembers: 2) settings.cluster.downingStrategy = .timeout(.init(downUnreachableMembersAfter: .milliseconds(200))) } @@ -263,6 +271,8 @@ final class ClusterLeaderActionsClusteredTests: ClusteredNodesTestBase { first.cluster.events.subscribe(p1.ref) let second = self.setUpNode("second") { settings in + settings.cluster.swim.probeInterval = .milliseconds(300) + settings.cluster.swim.pingTimeout = .milliseconds(100) settings.cluster.autoLeaderElection = .lowestReachable(minNumberOfMembers: 2) settings.cluster.downingStrategy = .timeout(.init(downUnreachableMembersAfter: .milliseconds(200))) } @@ -270,6 +280,8 @@ final class ClusterLeaderActionsClusteredTests: ClusteredNodesTestBase { second.cluster.events.subscribe(p2.ref) let third = self.setUpNode("third") { settings in + settings.cluster.swim.probeInterval = .milliseconds(300) + settings.cluster.swim.pingTimeout = .milliseconds(100) settings.cluster.autoLeaderElection = .lowestReachable(minNumberOfMembers: 2) settings.cluster.downingStrategy = .timeout(.init(downUnreachableMembersAfter: .milliseconds(200))) } @@ -286,19 +298,19 @@ final class ClusterLeaderActionsClusteredTests: ClusteredNodesTestBase { second.shutdown() // other nodes have observed it down - try self.ensureNodes(.down, on: first, nodes: second.cluster.node) - try self.ensureNodes(.down, on: third, nodes: second.cluster.node) + try self.ensureNodes(.down, on: first, within: .seconds(15), nodes: second.cluster.node) + try self.ensureNodes(.down, on: third, within: .seconds(15), nodes: second.cluster.node) // on the leader node, the other node noticed as up: let testKit = self.testKit(first) - try testKit.eventually(within: .seconds(10)) { + try testKit.eventually(within: .seconds(20)) { let event: Cluster.Event? = try p1.maybeExpectMessage() switch event { case .membershipChange(.init(node: second.cluster.node, fromStatus: .up, toStatus: .down)): () case let other: throw testKit.error("Expected `second` [ up] -> [ .down], on first node, was: \(other, orElse: "nil")") } } - try testKit.eventually(within: .seconds(10)) { + try testKit.eventually(within: .seconds(20)) { let event: Cluster.Event? = try p1.maybeExpectMessage() switch event { case .membershipChange(.init(node: second.cluster.node, fromStatus: .down, toStatus: .removed)): () diff --git a/Tests/DistributedActorsTests/Cluster/RemotingTLSClusteredTests.swift b/Tests/DistributedActorsTests/Cluster/RemotingTLSClusteredTests.swift index 3bc8b321a..3d5adadd9 100644 --- a/Tests/DistributedActorsTests/Cluster/RemotingTLSClusteredTests.swift +++ b/Tests/DistributedActorsTests/Cluster/RemotingTLSClusteredTests.swift @@ -190,6 +190,7 @@ class RemotingTLSTests: ClusteredNodesTestBase { let testKeySource2: NIOSSLPrivateKeySource = .privateKey(try NIOSSLPrivateKey(bytes: [UInt8](testKey2.utf8), format: .pem)) let local = self.setUpNode("local") { settings in + settings.cluster.node.host = "localhost" settings.cluster.tls = TLSConfiguration.forServer( certificateChain: [testCertificateSource1], privateKey: testKeySource1, @@ -199,6 +200,7 @@ class RemotingTLSTests: ClusteredNodesTestBase { } let remote = setUpNode("remote") { settings in + settings.cluster.node.host = "localhost" settings.cluster.tls = TLSConfiguration.forServer( certificateChain: [testCertificateSource2], privateKey: testKeySource2, @@ -265,7 +267,7 @@ class RemotingTLSTests: ClusteredNodesTestBase { let testCertificateSource: NIOSSLCertificateSource = .certificate(testCertificate) let testKey: NIOSSLPrivateKeySource = .privateKey(try NIOSSLPrivateKey(bytes: [UInt8](testKey1.utf8), format: .pem)) let local = self.setUpNode("local") { settings in - settings.cluster.node.host = "127.0.0.1" + settings.cluster.node.host = "localhost" settings.cluster.tls = TLSConfiguration.forServer( certificateChain: [testCertificateSource], privateKey: testKey, @@ -275,7 +277,7 @@ class RemotingTLSTests: ClusteredNodesTestBase { } let remote = setUpNode("remote") { settings in - settings.cluster.node.host = "127.0.0.1" + settings.cluster.node.host = "localhost" settings.cluster.tls = TLSConfiguration.forServer( certificateChain: [testCertificateSource], privateKey: testKey, diff --git a/Tests/DistributedActorsTests/Cluster/SWIM/SWIMShellClusteredTests.swift b/Tests/DistributedActorsTests/Cluster/SWIM/SWIMShellClusteredTests.swift index 7a934c3f6..d59babe05 100644 --- a/Tests/DistributedActorsTests/Cluster/SWIM/SWIMShellClusteredTests.swift +++ b/Tests/DistributedActorsTests/Cluster/SWIM/SWIMShellClusteredTests.swift @@ -55,9 +55,9 @@ final class SWIMShellClusteredTests: ClusteredNodesTestBase { "SWIM", SWIMShell.swimBehavior(members: [remoteProbeRef], clusterRef: self.firstClusterProbe.ref) { settings in settings.lifeguard.maxLocalHealthMultiplier = maxLocalHealthMultiplier - settings.failureDetector.pingTimeout = .microseconds(1) + settings.pingTimeout = .microseconds(1) // interval should be configured in a way that multiplied by a low LHA counter it will wail the test - settings.failureDetector.probeInterval = .milliseconds(100) + settings.probeInterval = .milliseconds(100) } ) @@ -90,9 +90,9 @@ final class SWIMShellClusteredTests: ClusteredNodesTestBase { "SWIM", SWIMShell.swimBehavior(members: [remoteProbeRef], clusterRef: self.firstClusterProbe.ref) { settings in settings.lifeguard.maxLocalHealthMultiplier = 1 - settings.failureDetector.pingTimeout = .microseconds(1) + settings.pingTimeout = .microseconds(1) // interval should be configured in a way that multiplied by a low LHA counter it will wail the test - settings.failureDetector.probeInterval = .milliseconds(100) + settings.probeInterval = .milliseconds(100) } ) @@ -116,9 +116,9 @@ final class SWIMShellClusteredTests: ClusteredNodesTestBase { "SWIM", SWIMShell.swimBehavior(members: [remoteProbeRef], clusterRef: self.firstClusterProbe.ref) { settings in settings.lifeguard.maxLocalHealthMultiplier = maxLocalHealthMultiplier - settings.failureDetector.pingTimeout = .milliseconds(500) + settings.pingTimeout = .milliseconds(500) // interval should be configured in a way that multiplied by a low LHA counter it will wail the test - settings.failureDetector.probeInterval = .milliseconds(100) + settings.probeInterval = .milliseconds(100) } ) @@ -270,7 +270,7 @@ final class SWIMShellClusteredTests: ClusteredNodesTestBase { let refB = try first.spawn("SWIMRefB", self.forwardingSWIMBehavior(forwardTo: probe.ref)) let behavior = SWIMShell.swimBehavior(members: [refA, refB], clusterRef: self.firstClusterProbe.ref) { settings in - settings.failureDetector.pingTimeout = .milliseconds(50) + settings.pingTimeout = .milliseconds(50) } let ref = try first.spawn("SWIM", behavior) @@ -300,7 +300,7 @@ final class SWIMShellClusteredTests: ClusteredNodesTestBase { let refB = try first.spawn("SWIMRefB", self.forwardingSWIMBehavior(forwardTo: probe.ref)) let behavior = SWIMShell.swimBehavior(members: [refA, refB], clusterRef: self.firstClusterProbe.ref) { settings in - settings.failureDetector.pingTimeout = .milliseconds(50) + settings.pingTimeout = .milliseconds(50) } let ref = try first.spawn("SWIM", behavior) @@ -374,7 +374,7 @@ final class SWIMShellClusteredTests: ClusteredNodesTestBase { settings.lifeguard.timeSourceNanos = timeSource.now settings.lifeguard.suspicionTimeoutMin = .nanoseconds(3) settings.lifeguard.suspicionTimeoutMax = .nanoseconds(6) - settings.failureDetector.pingTimeout = pingTimeout + settings.pingTimeout = pingTimeout } ) ) @@ -616,14 +616,14 @@ final class SWIMShellClusteredTests: ClusteredNodesTestBase { // unreachability event to the cluster upon such discovery. settings.cluster.swim.lifeguard.suspicionTimeoutMax = .seconds(100) settings.cluster.swim.lifeguard.suspicionTimeoutMin = .seconds(10) - settings.cluster.swim.failureDetector.pingTimeout = .seconds(3) + settings.cluster.swim.pingTimeout = .seconds(3) } let second = self.setUpSecond() let secondNode = second.cluster.node let third = self.setUpNode("third") { settings in settings.cluster.swim.lifeguard.suspicionTimeoutMin = .nanoseconds(2) settings.cluster.swim.lifeguard.suspicionTimeoutMax = .nanoseconds(2) - settings.cluster.swim.failureDetector.pingTimeout = .milliseconds(300) + settings.cluster.swim.pingTimeout = .milliseconds(300) } first.cluster.join(node: second.cluster.node.node) @@ -714,7 +714,6 @@ final class SWIMShellClusteredTests: ClusteredNodesTestBase { let remoteMemberRef = first._resolveKnownRemote(memberProbe.ref, onRemoteSystem: second) let swimRef = try first.spawn("SWIM", SWIMShell.swimBehavior(members: [remoteMemberRef], clusterRef: self.firstClusterProbe.ref)) - swimRef.tell(.remote(.ping(replyTo: remoteProbeRef, payload: .none))) let response: SWIM.PingResponse = try p.expectMessage() @@ -742,7 +741,7 @@ final class SWIMShellClusteredTests: ClusteredNodesTestBase { let remoteProbeRef = first._resolveKnownRemote(p.ref, onRemoteSystem: second) let behavior = SWIMShell.swimBehavior(members: [remoteProbeRef], clusterRef: self.firstClusterProbe.ref) { settings in - settings.failureDetector.pingTimeout = .milliseconds(50) + settings.pingTimeout = .milliseconds(50) } let swimRef = try first.spawn("SWIM", behavior) @@ -765,6 +764,41 @@ final class SWIMShellClusteredTests: ClusteredNodesTestBase { func test_swim_shouldSendGossipInPingReq() throws { let first = self.setUpFirst() +// +// let probe = self.testKit(first).spawnTestProbe(expecting: ForwardedSWIMMessage.self) +// +// let refA = try first.spawn("SWIM-A", self.forwardingSWIMBehavior(forwardTo: probe.ref)) +// let refB = try first.spawn("SWIM-B", self.forwardingSWIMBehavior(forwardTo: probe.ref)) +// +// let behavior = SWIMShell.swimBehavior(members: [refA, refB], clusterRef: self.firstClusterProbe.ref) { settings in +// settings.pingTimeout = .milliseconds(50) +// } +// +// let swimRef = try first.spawn("SWIM", behavior) +// +// swimRef.tell(.local(.pingRandomMember)) +// +// let forwardedPing = try probe.expectMessage() +// guard case SWIM.Message.remote(.ping(.alive(incarnation: 0), _, _)) = forwardedPing.message else { +// throw self.testKit(first).fail("Expected to receive `.ping`, got [\(forwardedPing.message)]") +// } +// let suspiciousRef = forwardedPing.recipient +// +// let forwardedPingReq = try probe.expectMessage() +// guard case SWIM.Message.remote(.pingReq(target: suspiciousRef, lastKnownStatus: .alive(0), _, let gossip)) = forwardedPingReq.message else { +// throw self.testKit(first).fail("Expected to receive `.pingReq` for \(suspiciousRef), got [\(forwardedPing.message)]") +// } +// +// switch gossip { +// case .membership(let members): +// members.shouldContain(SWIM.Member(ref: refA, status: .alive(incarnation: 0), protocolPeriod: 0)) +// members.shouldContain(SWIM.Member(ref: refB, status: .alive(incarnation: 0), protocolPeriod: 0)) +// members.shouldContain(SWIM.Member(ref: swimRef, status: .alive(incarnation: 0), protocolPeriod: 0)) +// members.count.shouldEqual(3) +// case .none: +// throw probe.error("Expected gossip, but got `.none`") +// } +// } let probe = self.testKit(first).spawnTestProbe(expecting: ForwardedSWIMMessage.self) @@ -881,7 +915,7 @@ final class SWIMShellClusteredTests: ClusteredNodesTestBase { let pingProbe = self.testKit(first).spawnTestProbe(expecting: SWIM.PingResponse.self) var settings: SWIMSettings = .default - settings.failureDetector.probeInterval = .milliseconds(100) + settings.probeInterval = .milliseconds(100) let firstSwim: ActorRef = try self.testKit(first)._eventuallyResolve(address: ._swim(on: first.cluster.node)) let secondSwim: ActorRef = try self.testKit(second)._eventuallyResolve(address: ._swim(on: second.cluster.node)) diff --git a/Tests/DistributedActorsTests/NodeDeathWatcherTests.swift b/Tests/DistributedActorsTests/NodeDeathWatcherTests.swift index cff7ff548..e7869c61c 100644 --- a/Tests/DistributedActorsTests/NodeDeathWatcherTests.swift +++ b/Tests/DistributedActorsTests/NodeDeathWatcherTests.swift @@ -21,10 +21,10 @@ final class NodeDeathWatcherTests: ClusteredNodesTestBase { func test_nodeDeath_shouldFailAllRefsOnSpecificAddress() throws { try shouldNotThrow { let first = self.setUpNode("first") { settings in - settings.cluster.swim.failureDetector.probeInterval = .milliseconds(100) + settings.cluster.swim.probeInterval = .milliseconds(100) } let second = self.setUpNode("second") { settings in - settings.cluster.swim.failureDetector.probeInterval = .milliseconds(100) + settings.cluster.swim.probeInterval = .milliseconds(100) } try self.joinNodes(node: first, with: second) diff --git a/Tests/DistributedActorsTests/SerializationTests.swift b/Tests/DistributedActorsTests/SerializationTests.swift index d58651a12..cd37fa256 100644 --- a/Tests/DistributedActorsTests/SerializationTests.swift +++ b/Tests/DistributedActorsTests/SerializationTests.swift @@ -117,7 +117,7 @@ class SerializationTests: ActorSystemTestBase { let addressAgain = try decoder.decode(ActorAddress.self, from: encoded) pinfo("Deserialized again: \(String(reflecting: addressAgain))") - "\(addressAgain)".shouldEqual("sact://SerializationTests@localhost:9001/user/hello") + "\(addressAgain)".shouldEqual("sact://SerializationTests@127.0.0.1:9001/user/hello") } } From 74e81903c3a995b4f0cce605465756cad0f75ba5 Mon Sep 17 00:00:00 2001 From: Konrad `ktoso` Malawski Date: Mon, 27 Apr 2020 23:18:38 +0900 Subject: [PATCH 05/18] =int-tests we're using 127.0.0.1 by default now --- ..._unreachable_unsuspend_causes_reachable.sh | 8 ++-- .../SWIM/SWIMShellClusteredTests.swift | 38 +------------------ 2 files changed, 5 insertions(+), 41 deletions(-) diff --git a/IntegrationTests/tests_04_cluster/test_01_suspend_causes_unreachable_unsuspend_causes_reachable.sh b/IntegrationTests/tests_04_cluster/test_01_suspend_causes_unreachable_unsuspend_causes_reachable.sh index 7815102c4..c4786cb2a 100755 --- a/IntegrationTests/tests_04_cluster/test_01_suspend_causes_unreachable_unsuspend_causes_reachable.sh +++ b/IntegrationTests/tests_04_cluster/test_01_suspend_causes_unreachable_unsuspend_causes_reachable.sh @@ -32,19 +32,19 @@ stdbuf -i0 -o0 -e0 swift run it_Clustered_swim_suspension_reachability 7337 > ${ declare -r first_pid=$(echo $!) wait_log_exists ${first_logs} 'Binding to: ' 200 # since it might be compiling again... -stdbuf -i0 -o0 -e0 swift run it_Clustered_swim_suspension_reachability 8228 localhost 7337 > ${second_logs} 2>&1 & +stdbuf -i0 -o0 -e0 swift run it_Clustered_swim_suspension_reachability 8228 127.0.0.1 7337 > ${second_logs} 2>&1 & declare -r second_pid=$(echo $!) wait_log_exists ${second_logs} 'Binding to: ' 200 # since it might be compiling again... echo "Waiting nodes to become .up..." -wait_log_exists ${first_logs} 'membershipChange(sact://System@localhost:8228 :: \[joining\] -> \[ up\])' 40 +wait_log_exists ${first_logs} 'membershipChange(sact://System@127.0.0.1:8228 :: \[joining\] -> \[ up\])' 50 echo 'Second member seen .up, good...' # suspend the second process, causing unreachability kill -SIGSTOP ${second_pid} jobs -wait_log_exists ${first_logs} 'reachabilityChange(DistributedActors.Cluster.ReachabilityChange.*localhost:8228, status: up, reachability: unreachable' 40 +wait_log_exists ${first_logs} 'reachabilityChange(DistributedActors.Cluster.ReachabilityChange.*127.0.0.1:8228, status: up, reachability: unreachable' 50 echo 'Second member seen .unreachable, good...' # resume it in the background @@ -52,7 +52,7 @@ kill -SIGCONT ${second_pid} # it should become reachable again declare -r expected_second_member_unreachable= -wait_log_exists ${first_logs} 'reachabilityChange(DistributedActors.Cluster.ReachabilityChange.*localhost:8228, status: up, reachability: reachable' 40 +wait_log_exists ${first_logs} 'reachabilityChange(DistributedActors.Cluster.ReachabilityChange.*127.0.0.1:8228, status: up, reachability: reachable' 50 echo 'Second member seen .unreachable, good...' diff --git a/Tests/DistributedActorsTests/Cluster/SWIM/SWIMShellClusteredTests.swift b/Tests/DistributedActorsTests/Cluster/SWIM/SWIMShellClusteredTests.swift index d59babe05..87d9c5c24 100644 --- a/Tests/DistributedActorsTests/Cluster/SWIM/SWIMShellClusteredTests.swift +++ b/Tests/DistributedActorsTests/Cluster/SWIM/SWIMShellClusteredTests.swift @@ -764,49 +764,13 @@ final class SWIMShellClusteredTests: ClusteredNodesTestBase { func test_swim_shouldSendGossipInPingReq() throws { let first = self.setUpFirst() -// -// let probe = self.testKit(first).spawnTestProbe(expecting: ForwardedSWIMMessage.self) -// -// let refA = try first.spawn("SWIM-A", self.forwardingSWIMBehavior(forwardTo: probe.ref)) -// let refB = try first.spawn("SWIM-B", self.forwardingSWIMBehavior(forwardTo: probe.ref)) -// -// let behavior = SWIMShell.swimBehavior(members: [refA, refB], clusterRef: self.firstClusterProbe.ref) { settings in -// settings.pingTimeout = .milliseconds(50) -// } -// -// let swimRef = try first.spawn("SWIM", behavior) -// -// swimRef.tell(.local(.pingRandomMember)) -// -// let forwardedPing = try probe.expectMessage() -// guard case SWIM.Message.remote(.ping(.alive(incarnation: 0), _, _)) = forwardedPing.message else { -// throw self.testKit(first).fail("Expected to receive `.ping`, got [\(forwardedPing.message)]") -// } -// let suspiciousRef = forwardedPing.recipient -// -// let forwardedPingReq = try probe.expectMessage() -// guard case SWIM.Message.remote(.pingReq(target: suspiciousRef, lastKnownStatus: .alive(0), _, let gossip)) = forwardedPingReq.message else { -// throw self.testKit(first).fail("Expected to receive `.pingReq` for \(suspiciousRef), got [\(forwardedPing.message)]") -// } -// -// switch gossip { -// case .membership(let members): -// members.shouldContain(SWIM.Member(ref: refA, status: .alive(incarnation: 0), protocolPeriod: 0)) -// members.shouldContain(SWIM.Member(ref: refB, status: .alive(incarnation: 0), protocolPeriod: 0)) -// members.shouldContain(SWIM.Member(ref: swimRef, status: .alive(incarnation: 0), protocolPeriod: 0)) -// members.count.shouldEqual(3) -// case .none: -// throw probe.error("Expected gossip, but got `.none`") -// } -// } - let probe = self.testKit(first).spawnTestProbe(expecting: ForwardedSWIMMessage.self) let refA = try first.spawn("SWIM-A", self.forwardingSWIMBehavior(forwardTo: probe.ref)) let refB = try first.spawn("SWIM-B", self.forwardingSWIMBehavior(forwardTo: probe.ref)) let behavior = SWIMShell.swimBehavior(members: [refA, refB], clusterRef: self.firstClusterProbe.ref) { settings in - settings.failureDetector.pingTimeout = .milliseconds(50) + settings.pingTimeout = .milliseconds(50) } let swimRef = try first.spawn("SWIM", behavior) From 6c1a39f4c4357f63aa2925aed7b711c623033173 Mon Sep 17 00:00:00 2001 From: Konrad `ktoso` Malawski Date: Tue, 28 Apr 2020 00:49:28 +0900 Subject: [PATCH 06/18] =test unlock test_singletonByClusterLeadership_withLeaderChange --- .../ActorSingletonPluginClusteredTests.swift | 6 ------ 1 file changed, 6 deletions(-) diff --git a/Tests/ActorSingletonPluginTests/ActorSingletonPluginClusteredTests.swift b/Tests/ActorSingletonPluginTests/ActorSingletonPluginClusteredTests.swift index 5b7e4013c..0d62aae3e 100644 --- a/Tests/ActorSingletonPluginTests/ActorSingletonPluginClusteredTests.swift +++ b/Tests/ActorSingletonPluginTests/ActorSingletonPluginClusteredTests.swift @@ -128,12 +128,6 @@ final class ActorSingletonPluginClusteredTests: ClusteredNodesTestBase { } func test_singletonByClusterLeadership_withLeaderChange() throws { - pnote("TODO: IGNORED UNTIL https://github.com/apple/swift-distributed-actors/issues/492 FIXED") - if Int.random(in: 10 ... 100) > 0 { - // trick to avoid getting a warning (which causes build failure under warnings-as-errors) - return () - } - try shouldNotThrow { var singletonSettings = ActorSingletonSettings(name: GreeterSingleton.name) singletonSettings.allocationStrategy = .byLeadership From ab2a941fe211590e158d9435a1a4497672e7d4ef Mon Sep 17 00:00:00 2001 From: Konrad `ktoso` Malawski Date: Tue, 28 Apr 2020 14:06:54 +0900 Subject: [PATCH 07/18] =remote protect association from being completed multiple times --- Sources/DistributedActors/Behaviors.swift | 7 +- .../Cluster/Association.swift | 57 +++++- .../Cluster/ClusterShell.swift | 164 ++++++++++-------- .../Cluster/ClusterShellState.swift | 45 +++-- .../Cluster/HandshakeStateMachine.swift | 22 ++- .../Cluster/SWIM/SWIMShell.swift | 69 ++++---- .../Transport/TransportPipelines.swift | 13 +- .../Cluster/Transport/WireMessages.swift | 7 +- .../DistributedActors/String+Extensions.swift | 6 +- .../ActorPingPongBenchmarks.swift | 9 +- .../ActorRemotePingPongBenchmarks.swift | 28 ++- .../ActorSingletonPluginClusteredTests.swift | 14 +- .../Cluster/AssociationClusteredTests.swift | 35 +++- .../DowningClusteredTests.swift | 119 +++++++------ 14 files changed, 391 insertions(+), 204 deletions(-) diff --git a/Sources/DistributedActors/Behaviors.swift b/Sources/DistributedActors/Behaviors.swift index 2bd9cce53..56bc1693f 100644 --- a/Sources/DistributedActors/Behaviors.swift +++ b/Sources/DistributedActors/Behaviors.swift @@ -483,7 +483,12 @@ public extension Behavior { case .suspend: fatalError("Illegal to attempt to interpret message with .suspend behavior! Behavior should have been canonicalized. This is a bug, please open a ticket.", file: file, line: line) case .suspended: - fatalError("No message should ever be delivered to a .suspended behavior! This is a bug, please open a ticket.", file: file, line: line) + fatalError(""" + No message should ever be delivered to a .suspended behavior! + Message: \(message) + Actor: \(context) + This is a bug, please open a ticket. + """, file: file, line: line) } } diff --git a/Sources/DistributedActors/Cluster/Association.swift b/Sources/DistributedActors/Cluster/Association.swift index 3cc16b75a..1e0ea09d2 100644 --- a/Sources/DistributedActors/Cluster/Association.swift +++ b/Sources/DistributedActors/Cluster/Association.swift @@ -62,7 +62,12 @@ final class Association: CustomStringConvertible { /// Complete the association and drain any pending message sends onto the channel. // TODO: This style can only ever work since we lock around the entirety of enqueueing messages and this setting; make it such that we don't need the lock eventually func completeAssociation(handshake: HandshakeStateMachine.CompletedState, over channel: Channel) { - // TODO: assert that the channel is for the right remote node? + assert(self.remoteNode == handshake.remoteNode, + """ + Complete association with wrong node was invoked. \ + Association, remote node: \(self.remoteNode); \ + Handshake, remote node: \(handshake.remoteNode) + """) self.lock.withLockVoid { switch self.state { @@ -78,12 +83,11 @@ final class Association: CustomStringConvertible { _ = channel.writeAndFlush(envelope) } - case .associated(let existingAssociatedChannel): - fatalError("MUST NOT complete an association twice; Was \(existingAssociatedChannel) and tried to complete with \(channel) from \(handshake)") + case .associated: + _ = channel.close() case .tombstone: _ = channel.close() - return } } } @@ -118,6 +122,19 @@ final class Association: CustomStringConvertible { var description: String { "AssociatedState(\(self.state), selfNode: \(self.selfNode), remoteNode: \(self.remoteNode))" } + + func whenComplete(_ promise: EventLoopPromise?) { + self.lock.withLockVoid { + switch self.state { + case .associating: + () // TODO trigger when it completes + case .associated: + () // promise?.succeed(.accept(HandshakeAccept())) + case .tombstone: + promise?.fail(HandshakeError.targetAlreadyTombstone(selfNode: self.selfNode, remoteNode: self.remoteNode)) + } + } + } } extension Association { @@ -139,16 +156,48 @@ extension Association { self.lock.withLockVoid { switch self.state { case .associating(let sendQueue): + pprint("SEND [ENQUEUE] = \(envelope.underlyingMessage) >>>> \(envelope.recipient)") sendQueue.enqueue(envelope) case .associated(let channel): + // pprint("SEND [SEND] = \(envelope.underlyingMessage) >>>> \(envelope.recipient)") channel.writeAndFlush(envelope, promise: promise) case .tombstone(let deadLetters): + pprint("SEND [DEAD] = \(envelope.underlyingMessage) >>>> \(envelope.recipient)") deadLetters.tell(.init(envelope.underlyingMessage, recipient: envelope.recipient)) } } } } +extension Association { + public var isAssociating: Bool { + switch self.state { + case .associating: + return true + default: + return false + } + } + + public var isAssociated: Bool { + switch self.state { + case .associated: + return true + default: + return false + } + } + + public var isTombstone: Bool { + switch self.state { + case .tombstone: + return true + default: + return false + } + } +} + // ==== ---------------------------------------------------------------------------------------------------------------- // MARK: Association Tombstone diff --git a/Sources/DistributedActors/Cluster/ClusterShell.swift b/Sources/DistributedActors/Cluster/ClusterShell.swift index 0405a236f..49e6125ce 100644 --- a/Sources/DistributedActors/Cluster/ClusterShell.swift +++ b/Sources/DistributedActors/Cluster/ClusterShell.swift @@ -43,7 +43,7 @@ internal class ClusterShell { /// Used by remote actor refs to obtain associations /// - Protected by: `_associationsLock` - private var _activeAssociations: [UniqueNode: Association] + private var _associations: [UniqueNode: Association] /// Node tombstones are kept here in order to avoid attempting to associate if we get a reference to such node, /// which would normally trigger an `ensureAssociated`. /// - Protected by: `_associationsLock` @@ -53,7 +53,7 @@ internal class ClusterShell { self._associationsLock.withLock { // TODO: a bit terrible; perhaps we should key be Node and then confirm by UniqueNode? // this used to be separated in the State keeping them by Node and here we kept by unique though that caused other challenges - self._activeAssociations.first { + self._associations.first { key, _ in key.node == node }?.value } @@ -65,11 +65,11 @@ internal class ClusterShell { self._associationsLock.withLock { if let tombstone = self._associationTombstones[node] { return .tombstone(tombstone) - } else if let existing = self._activeAssociations[node] { + } else if let existing = self._associations[node] { return .association(existing) } else { let association = Association(selfNode: self.selfNode, remoteNode: node) - self._activeAssociations[node] = association + self._associations[node] = association /// We're trying to send to `node` yet it has no association (not even in progress), /// thus we need to kick it off. Once it completes it will .completeAssociation() on the stored one (here in the field in Shell). @@ -87,16 +87,16 @@ internal class ClusterShell { case tombstone(Association.Tombstone) } - /// To be invoked by cluster shell whenever an association is made. + /// To be invoked by cluster shell whenever handshake is accepted, creating a completed association. /// Causes messages to be flushed onto the new associated channel. private func storeCompleteAssociation(_ associated: ClusterShellState.AssociatedDirective) { self._associationsLock.withLockVoid { - let association = self._activeAssociations[associated.handshake.remoteNode] ?? + let association = self._associations[associated.handshake.remoteNode] ?? Association(selfNode: associated.handshake.localNode, remoteNode: associated.handshake.remoteNode) association.completeAssociation(handshake: associated.handshake, over: associated.channel) - self._activeAssociations[associated.handshake.remoteNode] = association + self._associations[associated.handshake.remoteNode] = association } } @@ -114,7 +114,7 @@ internal class ClusterShell { // No more message sends to the system will be possible. traceLog_Remote(system.cluster.node, "Finish terminate association [\(remoteNode)]: Stored tombstone") self._associationTombstones[remoteNode] = Association.Tombstone(remoteNode, settings: system.settings.cluster) - return self._activeAssociations.removeValue(forKey: remoteNode) + return self._associations.removeValue(forKey: remoteNode) } guard let removedAssociation = removedAssociationOption else { @@ -180,14 +180,14 @@ internal class ClusterShell { /// Safe to concurrently access by privileged internals. internal var _testingOnly_associations: [Association] { self._associationsLock.withLock { - [Association](self._activeAssociations.values) + [Association](self._associations.values) } } /// For testing only. internal func _associatedNodes() -> Set { self._associationsLock.withLock { - Set(self._activeAssociations.keys) + Set(self._associations.keys) } } @@ -225,7 +225,7 @@ internal class ClusterShell { init(selfNode: UniqueNode) { self.selfNode = selfNode self._associationsLock = Lock() - self._activeAssociations = [:] + self._associations = [:] self._associationTombstones = [:] // not enjoying this dance, but this way we can share the ClusterShell as the shell AND the container for the ref. @@ -473,7 +473,7 @@ extension ClusterShell { switch message { case .handshakeOffer(let offer, let channel, let promise): self.tracelog(context, .receiveUnique(from: offer.from), message: offer) - return self.onHandshakeOffer(context, state, offer, channel: channel, replyInto: promise) + return self.onHandshakeOffer(context, state, offer, incomingChannel: channel, replyInto: promise) case .handshakeAccepted(let accepted, let channel): self.tracelog(context, .receiveUnique(from: accepted.from), message: accepted) @@ -617,10 +617,13 @@ extension ClusterShell { // if an association exists for any UniqueNode that this Node represents, we can use this and abort the handshake dance here if let existingAssociation = self.getExistingAssociation(with: remoteNode) { - state.log.debug("Attempted associating with already associated node: \(reflecting: remoteNode), existing association: [\(existingAssociation)]") + state.log.debug("Association already allocated for remote: \(reflecting: remoteNode), existing association: [\(existingAssociation)]") switch existingAssociation.state { - case .associating: - () // ok, continue the association dance + case .associating(_): +// whenComplete.futureResult.whenComplete { _ in +// replyTo?.tell() +// } + () // continue, we may be the first beginHandshake (as associations may be ensured outside of actor context) case .associated: // return , we've been successful already replyTo?.tell(.success(existingAssociation.remoteNode)) @@ -640,15 +643,7 @@ extension ClusterShell { whenHandshakeComplete.futureResult.whenComplete { result in switch result { case .success(.accept(let accept)): - /// we need to switch here, since we MAY have been attached to an ongoing handshake which may have been initiated - /// in either direction // TODO check if this is really needed. - let associatedRemoteNode: UniqueNode - if accept.from.node == remoteNode { - associatedRemoteNode = accept.from - } else { - associatedRemoteNode = accept.origin - } - replyTo?.tell(.success(associatedRemoteNode)) + replyTo?.tell(.success(accept.origin)) case .success(.reject(let reject)): replyTo?.tell(.failure(HandshakeConnectionError(node: remoteNode, message: reject.reason))) case .failure(let error): @@ -662,9 +657,11 @@ extension ClusterShell { switch handshakeState { case .initiated(let initiated): + state.log.info("Initiated handshake: \(initiated)") return self.connectSendHandshakeOffer(context, state, initiated: initiated) case .wasOfferedHandshake, .inFlight, .completed: + state.log.info("Handshake in other state: \(handshakeState)") // the reply will be handled already by the future.whenComplete we've set up above here // so nothing to do here, just become the next state return self.ready(state: state) @@ -674,6 +671,15 @@ extension ClusterShell { func connectSendHandshakeOffer(_ context: ActorContext, _ state: ClusterShellState, initiated: HandshakeStateMachine.InitiatedState) -> Behavior { var state = state +// if let existingAssociation = self.getExistingAssociation(with: initiated.remoteNode) { +// state.log.info("No need to extend handshake offer, node [\(existingAssociation.remoteNode)] is already associated.", metadata: [ +// "handshake": "\(existingAssociation.state)" +// ]) +// existingAssociation.whenComplete(initiated.whenCompleted) +// // TODO: ensure state has it as well; clear the handshake +// return self.ready(state: state) +// } + state.log.info("Extending handshake offer to \(initiated.remoteNode))") // TODO: log retry stats? let offer: Wire.HandshakeOffer = initiated.makeOffer() @@ -708,21 +714,25 @@ extension ClusterShell { extension ClusterShell { /// Initial entry point for accepting a new connection; Potentially allocates new handshake state machine. internal func onHandshakeOffer( - _ context: ActorContext, _ state: ClusterShellState, - _ offer: Wire.HandshakeOffer, channel: Channel, - replyInto promise: EventLoopPromise + _ context: ActorContext, _ state: ClusterShellState, + _ offer: Wire.HandshakeOffer, incomingChannel: Channel, + replyInto promise: EventLoopPromise ) -> Behavior { var state = state - switch state.onIncomingHandshakeOffer(offer: offer) { - case .negotiate(let hsm): + state.log.notice("HANDLING ON HANDSHAKE OFFER \(offer) >>> \(incomingChannel)!") + + switch state.onIncomingHandshakeOffer(offer: offer, incomingChannel: incomingChannel) { + case .negotiateIncoming(let hsm): // handshake is allowed to proceed switch hsm.negotiate() { case .acceptAndAssociate(let completedHandshake): - state.log.info("Accept association with \(reflecting: offer.from)!") + state.log.info("Accept association with \(reflecting: offer.from)!", metadata: [ + "handshake/channel": "\(incomingChannel)" + ]) - // create and store association - let directive = state.associate(context.system, completedHandshake, channel: channel) + // accept handshake and store completed association + let directive = state.associate(context.system, completedHandshake, channel: incomingChannel) self.storeCompleteAssociation(directive) // send accept to other node @@ -730,24 +740,9 @@ extension ClusterShell { self.tracelog(context, .send(to: offer.from.node), message: accept) promise.succeed(.accept(accept)) - // TODO: This is a bit duplicated // This association may mean that we've "replaced" a previously known node of the same host:port, // In case of such replacement we must down and terminate the association of the previous node. - if let replacedMember = directive.membershipChange.replaced { - // the change was a replacement and thus we need to down the old member (same host:port as the new one), - // and terminate its association. - - state.log.info("Accepted handshake from [\(accept.from)] which replaces the previously known: \(reflecting: replacedMember).") - - // We MUST be careful to first terminate the association and then store the new one in 2) - self.terminateAssociation(context.system, state: &state, replacedMember.node) - - // By emitting these `change`s, we not only let anyone interested know about this, - // but we also enable the shell (or leadership) to update the leader if it needs changing. - // - // We MUST emit this `.down` before emitting the replacement's event - state.events.publish(.membershipChange(.init(member: replacedMember, toStatus: .down))) - } + self.handlePotentialAssociatedMemberReplacement(directive: directive, accept: accept, context: context, state: &state) // publish any cluster events this association caused. // As the new association is stored, any reactions to these events will use the right underlying connection @@ -775,7 +770,7 @@ extension ClusterShell { return self.ready(state: state) } - case .abortDueToConcurrentHandshake: + case .abortIncomingDueToConcurrentHandshake: // concurrent handshake and we should abort let error = HandshakeConnectionError( node: offer.from.node, @@ -784,7 +779,7 @@ extension ClusterShell { which will be used to complete the handshake. """ ) - state.abortIncomingHandshake(offer: offer, channel: channel) + state.closeHandshakeChannel(offer: offer, channel: incomingChannel) promise.fail(error) return .same } @@ -815,7 +810,7 @@ extension ClusterShell { delay: delay ) case .giveUpOnHandshake: - if let hsmState = state.abortOutgoingHandshake(with: remoteNode) { + if let hsmState = state.closeOutboundHandshakeChannel_TODOBETTERNAME(with: remoteNode) { self.notifyHandshakeFailure(state: hsmState, node: remoteNode, error: error) } } @@ -854,21 +849,7 @@ extension ClusterShell { // 1.1) This association may mean that we've "replaced" a previously known node of the same host:port, // In case of such replacement we must down and terminate the association of the previous node. - if let replacedMember = directive.membershipChange.replaced { - // the change was a replacement and thus we need to down the old member (same host:port as the new one), - // and terminate its association. - - state.log.info("Accepted handshake from [\(accept.from)] which replaces the previously known: \(reflecting: replacedMember).") - - // We MUST be careful to first terminate the association and then store the new one in 2) - self.terminateAssociation(context.system, state: &state, replacedMember.node) - - // By emitting these `change`s, we not only let anyone interested know about this, - // but we also enable the shell (or leadership) to update the leader if it needs changing. - // - // We MUST emit this `.down` before emitting the replacement's event - state.events.publish(.membershipChange(.init(member: replacedMember, toStatus: .down))) - } + self.handlePotentialAssociatedMemberReplacement(directive: directive, accept: accept, context: context, state: &state) // 2) Store the (now completed) association first, as it may be immediately used by remote ActorRefs attempting to send to the remoteNode self.storeCompleteAssociation(directive) // we MUST store it outside the state @@ -887,10 +868,37 @@ extension ClusterShell { self.recordMetrics(context.system.metrics, membership: state.membership) // 5) Finally, signal the handshake future that we've accepted, and become with ready state + print("handshakeCompleted.makeAccept() === \(handshakeCompleted.makeAccept())") + print(" accept === \(accept)") handshakeCompleted.whenCompleted?.succeed(.accept(handshakeCompleted.makeAccept())) return self.ready(state: state) } + /// An accept may imply that it replaced a previously associated member. + /// If so, this method will .down it in the membership and terminate the previous instances association. + private func handlePotentialAssociatedMemberReplacement( + directive: ClusterShellState.AssociatedDirective, + accept: Wire.HandshakeAccept, + context: ActorContext, + state: inout ClusterShellState + ) { + if let replacedMember = directive.membershipChange.replaced { + // the change was a replacement and thus we need to down the old member (same host:port as the new one), + // and terminate its association. + + state.log.info("Accepted handshake from [\(accept.from)] which replaces the previously known: \(reflecting: replacedMember).") + + // We MUST be careful to first terminate the association and then store the new one in 2) + self.terminateAssociation(context.system, state: &state, replacedMember.node) + + // By emitting these `change`s, we not only let anyone interested know about this, + // but we also enable the shell (or leadership) to update the leader if it needs changing. + // + // We MUST emit this `.down` before emitting the replacement's event + state.events.publish(.membershipChange(.init(member: replacedMember, toStatus: .down))) + } + } + private func onHandshakeRejected(_ context: ActorContext, _ state: ClusterShellState, _ reject: Wire.HandshakeReject) -> Behavior { var state = state @@ -898,7 +906,7 @@ extension ClusterShell { // TODO: back off intensely, give up after some attempts? - if let hsmState = state.abortOutgoingHandshake(with: reject.from) { + if let hsmState = state.closeOutboundHandshakeChannel_TODOBETTERNAME(with: reject.from) { self.notifyHandshakeFailure(state: hsmState, node: reject.from, error: HandshakeConnectionError(node: reject.from, message: reject.reason)) } @@ -907,14 +915,24 @@ extension ClusterShell { } private func onHandshakeFailed(_ context: ActorContext, _ state: ClusterShellState, with node: Node, error: Error) -> Behavior { - var state = state - - state.log.error("Handshake error while connecting [\(node)]: \(error)") - if let hsmState = state.abortOutgoingHandshake(with: node) { - self.notifyHandshakeFailure(state: hsmState, node: node, error: error) +// var state = state + + // we MAY be seeing a handshake failure from a 2 nodes concurrently shaking hands on 2 connections, + // and we decided to tie-break and kill one of the connections. As such, the handshake COMPLETED successfully but + // on the other connection; and the terminated one may yield an error (e.g. truncation error during proto parsing etc), + // however that error is harmless - as we associated with the "other" right connection. + if let existingAssociation = self.getExistingAssociation(with: node), + existingAssociation.isAssociated || existingAssociation.isTombstone { + state.log.trace("Handshake failed, however existing association with node exists. Could be that a concurrent handshake was failed on purpose.", + metadata: state.metadataForHandshakes(node: node, error: error)) + return .same } - self.recordMetrics(context.system.metrics, membership: state.membership) +// state.log.error("Handshake error while connecting [\(node)]: \(error)", metadata: state.metadataForHandshakes(node: node, error: error)) +// if let hsmState = state.closeOutboundHandshakeChannel_TODOBETTERNAME(with: node) { +// self.notifyHandshakeFailure(state: hsmState, node: node, error: error) +// } + return self.ready(state: state) } diff --git a/Sources/DistributedActors/Cluster/ClusterShellState.swift b/Sources/DistributedActors/Cluster/ClusterShellState.swift index 69f2bcfad..4da4dff55 100644 --- a/Sources/DistributedActors/Cluster/ClusterShellState.swift +++ b/Sources/DistributedActors/Cluster/ClusterShellState.swift @@ -180,14 +180,16 @@ extension ClusterShellState { /// /// - Faults: when called in wrong state of an ongoing handshake /// - Returns: if present, the (now removed) handshake state that was aborted, hil otherwise. - mutating func abortOutgoingHandshake(with node: Node) -> HandshakeStateMachine.State? { + // THIS SIGNATURE IS SCARY; dont kill a Node, kill a specific channel (!) + mutating func closeOutboundHandshakeChannel_TODOBETTERNAME(with node: Node, file: String = #file, line: UInt = #line) -> HandshakeStateMachine.State? { guard let state = self._handshakes.removeValue(forKey: node) else { return nil } - switch state { case .initiated(let initiated): if let channel = initiated.channel { + self.log.warning("ABORTING OUTBOUND handshake channel: \(channel) [AT: \(file):\(line)]") + channel.close().whenFailure { [log = self.log] error in log.warning("Failed to abortOutgoingHandshake (close) channel [\(channel)], error: \(error)") } @@ -208,7 +210,9 @@ extension ClusterShellState { /// by node from the _handshakes. /// /// - Returns: if present, the (now removed) handshake state that was aborted, hil otherwise. - mutating func abortIncomingHandshake(offer: Wire.HandshakeOffer, channel: Channel) { + mutating func closeHandshakeChannel(offer: Wire.HandshakeOffer, channel: Channel) { + self.log.warning("ABORTING INBOUND handshake channel: \(channel)") + channel.close().whenFailure { [log = self.log, metadata = self.metadata] error in log.warning("Failed to abortIncomingHandshake (close) channel [\(channel)], error: \(error)", metadata: metadata) } @@ -228,12 +232,12 @@ extension ClusterShellState { /// Upon tie-break the nodes follow these two roles: /// Winner: Keeps the outgoing connection, negotiates and replies accept/reject on the "incoming" connection from the remote node. /// Loser: Drops the incoming connection and waits for Winner's decision. - mutating func onIncomingHandshakeOffer(offer: Wire.HandshakeOffer) -> OnIncomingHandshakeOfferDirective { + mutating func onIncomingHandshakeOffer(offer: Wire.HandshakeOffer, incomingChannel: Channel) -> OnIncomingHandshakeOfferDirective { func negotiate(promise: EventLoopPromise? = nil) -> OnIncomingHandshakeOfferDirective { let promise = promise ?? self.eventLoopGroup.next().makePromise(of: Wire.HandshakeResponse.self) let fsm = HandshakeStateMachine.HandshakeReceivedState(state: self, offer: offer, whenCompleted: promise) self._handshakes[offer.from.node] = .wasOfferedHandshake(fsm) - return .negotiate(fsm) + return .negotiateIncoming(fsm) } guard let inProgress = self._handshakes[offer.from.node] else { @@ -243,14 +247,26 @@ extension ClusterShellState { switch inProgress { case .initiated(let initiated): + + /// Since we MAY have 2 connections open at this point in time -- one we opened, and another that was opened + /// to us when the other node tried to associated, we'll perform a tie-breaker to ensure we predictably + /// only use _one_ of them, and close the other. + // let selectedChannel: Channel + /// order on nodes is somewhat arbitrary, but that is fine, since we only need this for tiebreakers let tieBreakWinner = initiated.localNode < offer.from self.log.info(""" Concurrently initiated handshakes from nodes [\(initiated.localNode)](local) and [\(offer.from)](remote) \ detected! Resolving race by address ordering; This node \(tieBreakWinner ? "WON (will negotiate and reply)" : "LOST (will await reply)") tie-break. - """) + """, metadata: [ + "handshake/inProgress": "\(initiated)", + "handshake/incoming/offer": "\(offer)", + "handshake/incoming/channel": "\(incomingChannel)", + ]) + if tieBreakWinner { - if self.abortOutgoingHandshake(with: offer.from.node) != nil { + + if self.closeOutboundHandshakeChannel_TODOBETTERNAME(with: offer.from.node) != nil { self.log.debug( "Aborted handshake, as concurrently negotiating another one with same node already", metadata: [ @@ -265,7 +281,7 @@ extension ClusterShellState { } else { // we "lost", the other node will send the accept; when it does, the will complete the future. - return .abortDueToConcurrentHandshake + return .abortIncomingDueToConcurrentHandshake } case .wasOfferedHandshake: // suspicious but but not wrong, so we were offered before, and now are being offered again? @@ -283,10 +299,10 @@ extension ClusterShellState { } enum OnIncomingHandshakeOfferDirective { - case negotiate(HandshakeStateMachine.HandshakeReceivedState) + case negotiateIncoming(HandshakeStateMachine.HandshakeReceivedState) /// An existing handshake with given peer is already in progress, /// do not negotiate but rest assured that the association will be handled properly by the already ongoing process. - case abortDueToConcurrentHandshake + case abortIncomingDueToConcurrentHandshake } mutating func incomingHandshakeAccept(_ accept: Wire.HandshakeAccept) -> HandshakeStateMachine.CompletedState? { @@ -425,4 +441,13 @@ extension ClusterShellState { "membership/count": "\(String(describing: self.membership.count(atLeast: .joining)))", ] } + + func metadataForHandshakes(node: Node, error: Error) -> Logger.Metadata { + [ + "handshake/error": "\(error)", + "handshake/errorType": "\(String(reflecting: type(of: error as Any)))", + "handshake/peer": "\(node)", + "handshakes": "\(self.handshakes())" + ] + } } diff --git a/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift b/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift index 5ace78571..2ddd3cc19 100644 --- a/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift +++ b/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift @@ -60,11 +60,11 @@ internal struct HandshakeStateMachine { // ==== ------------------------------------------------------------------------------------------------------------ // MARK: Handshake Initiated - internal struct InitiatedState { + internal struct InitiatedState: Swift.CustomStringConvertible { var backoff: BackoffStrategy let settings: ClusterSettings - var protocolVersion: DistributedActors.Version { + var protocolVersion: Version { self.settings.protocolVersion } @@ -97,7 +97,7 @@ internal struct HandshakeStateMachine { Wire.HandshakeOffer(version: self.protocolVersion, from: self.localNode, to: self.remoteNode) } - mutating func onHandshakeTimeout() -> HandshakeStateMachine.RetryDirective { + mutating func onHandshakeTimeout() -> RetryDirective { if let interval = self.backoff.next() { return .scheduleRetryHandshake(delay: interval) } else { @@ -109,7 +109,7 @@ internal struct HandshakeStateMachine { self.channel = channel } - mutating func onHandshakeError(_: Error) -> HandshakeStateMachine.RetryDirective { + mutating func onHandshakeError(_: Error) -> RetryDirective { switch self.backoff.next() { case .some(let amount): return .scheduleRetryHandshake(delay: amount) @@ -117,6 +117,18 @@ internal struct HandshakeStateMachine { return .giveUpOnHandshake } } + + var description: Swift.String { + """ + InitiatedState(\ + remoteNode: \(remoteNode), \ + localNode: \(localNode), \ + backoff: \(backoff), \ + whenCompleted: \(optional: whenCompleted), \ + channel: \(optional: channel)\ + ) + """ + } } // // ==== ------------------------------------------------------------------------------------------------------------ @@ -296,4 +308,6 @@ enum HandshakeError: Error { /// Returned when an incoming handshake protocol version does not match what this node can understand. case incompatibleProtocolVersion(local: DistributedActors.Version, remote: DistributedActors.Version) + + case targetAlreadyTombstone(selfNode: UniqueNode, remoteNode: UniqueNode) } diff --git a/Sources/DistributedActors/Cluster/SWIM/SWIMShell.swift b/Sources/DistributedActors/Cluster/SWIM/SWIMShell.swift index 072e214b5..9038205d2 100644 --- a/Sources/DistributedActors/Cluster/SWIM/SWIMShell.swift +++ b/Sources/DistributedActors/Cluster/SWIM/SWIMShell.swift @@ -102,7 +102,7 @@ internal struct SWIMShell { self.processGossipPayload(context: context, payload: payload) if !self.swim.isMember(target) { - self.ensureAssociated(context, remoteNode: target.address.node) { result in + self.withEnsuredAssociation(context, remoteNode: target.address.node) { result in switch result { case .success: // The case when member is a suspect is already handled in `processGossipPayload`, since @@ -447,7 +447,7 @@ internal struct SWIMShell { switch self.swim.onGossipPayload(about: member) { case .connect(let node, let continueAddingMember): // ensuring a connection is asynchronous, but executes callback in actor context - self.ensureAssociated(context, remoteNode: node) { uniqueAddressResult in + self.withEnsuredAssociation(context, remoteNode: node) { uniqueAddressResult in switch uniqueAddressResult { case .success(let uniqueAddress): continueAddingMember(.success(uniqueAddress)) @@ -512,45 +512,46 @@ internal struct SWIMShell { self.clusterRef.tell(.command(.failureDetectorReachabilityChanged(change.member.node, reachability))) } - /// Use to ensure an association to given remote node exists; as one may not always be sure a connection has been already established, - /// when a remote ref is discovered through other means (such as SWIM's gossiping). - func ensureAssociated(_ context: ActorContext, remoteNode: UniqueNode?, continueWithAssociation: @escaping (Result) -> Void) { + // TODO: remove this + /// Use to ensure an association to given remote node exists. + func withEnsuredAssociation(_ context: ActorContext, remoteNode: UniqueNode?, continueWithAssociation: @escaping (Result) -> Void) { // this is a local node, so we don't need to connect first guard let remoteNode = remoteNode else { continueWithAssociation(.success(context.system.cluster.node)) return } - guard let clusterShell = context.system._cluster else { - continueWithAssociation(.failure(EnsureAssociationError("ClusterShell not available when trying to ensure associated with: \(reflecting: remoteNode)"))) - return - } - - let associationState = clusterShell.getEnsureAssociation(with: remoteNode) - switch associationState { - case .association(let control): - continueWithAssociation(.success(control.remoteNode)) - case .tombstone: - let msg = "Association target node is already .tombstoned, not associating. Node \(reflecting: remoteNode) likely to be removed from gossip shortly." - continueWithAssociation(.failure(EnsureAssociationError(msg))) - return // we shall not associate with this tombstoned node (!) - } - - // ensure connection to new node ~~~ - // TODO: might need a cache in the swim shell? // actual solution being a shared concurrent hashmap... - // FIXME: use reasonable timeout and back off? issue #141 - let ref = context.messageAdapter(from: ClusterShell.HandshakeResult.self) { (result: ClusterShell.HandshakeResult) in - switch result { - case .success(let uniqueNode): - return SWIM.Message.local(.monitor(uniqueNode)) - case .failure(let error): - context.log.debug("Did not associate with \(reflecting: remoteNode), reason: \(error)") - return nil // drop the message - } - } + // actor refs handle kicking off associations automatically when attempting to send to them; so we do nothing here (!!!) + continueWithAssociation(.success(remoteNode)) - context.log.trace("Requesting handshake with \(remoteNode.node)") - self.clusterRef.tell(.command(.handshakeWith(remoteNode.node, replyTo: ref))) +// guard let clusterShell = context.system._cluster else { +// continueWithAssociation(.failure(EnsureAssociationError("ClusterShell not available when trying to ensure associated with: \(reflecting: remoteNode)"))) +// return +// } +// +// let associationState = clusterShell.getEnsureAssociation(with: remoteNode) +// switch associationState { +// case .association(let control): +// continueWithAssociation(.success(control.remoteNode)) +// case .tombstone: +// let msg = "Association target node is already .tombstoned, not associating. Node \(reflecting: remoteNode) likely to be removed from gossip shortly." +// continueWithAssociation(.failure(EnsureAssociationError(msg))) +// return // we shall not associate with this tombstoned node (!) +// } +// +// // ensure connection to new node ~~~ +// let ref = context.messageAdapter(from: ClusterShell.HandshakeResult.self) { (result: ClusterShell.HandshakeResult) in +// switch result { +// case .success(let uniqueNode): +// return SWIM.Message.local(.monitor(uniqueNode)) +// case .failure(let error): +// context.log.debug("Did not associate with \(reflecting: remoteNode), reason: \(error)") +// return nil // drop the message +// } +// } +// +// context.log.trace("Requesting handshake with \(remoteNode.node)") +// self.clusterRef.tell(.command(.handshakeWith(remoteNode.node, replyTo: ref))) } struct EnsureAssociationError: Error { diff --git a/Sources/DistributedActors/Cluster/Transport/TransportPipelines.swift b/Sources/DistributedActors/Cluster/Transport/TransportPipelines.swift index fd83140dd..07632f1ba 100644 --- a/Sources/DistributedActors/Cluster/Transport/TransportPipelines.swift +++ b/Sources/DistributedActors/Cluster/Transport/TransportPipelines.swift @@ -52,22 +52,27 @@ private final class InitiatingHandshakeHandler: ChannelInboundHandler, Removable func channelRead(context: ChannelHandlerContext, data: NIOAny) { var bytes = self.unwrapInboundIn(data) + let metadata: Logger.Metadata = [ + "handshake/channel": "\(context.channel)" + ] + do { let response = try self.readHandshakeResponse(bytes: &bytes) switch response { case .accept(let accept): - self.log.debug("Received handshake accept from: [\(accept.from)]") + self.log.debug("Received handshake accept from: [\(accept.from)]", metadata: metadata) self.cluster.tell(.inbound(.handshakeAccepted(accept, channel: context.channel))) // handshake is completed, so we remove the handler from the pipeline context.pipeline.removeHandler(self, promise: nil) case .reject(let reject): - self.log.debug("Received handshake reject from: [\(reject.from)] reason: [\(reject.reason)]") + self.log.debug("Received handshake reject from: [\(reject.from)] reason: [\(reject.reason)], closing channel.", metadata: metadata) self.cluster.tell(.inbound(.handshakeRejected(reject))) context.close(promise: nil) } } catch { + self.log.debug("Handshake failure, error [\(error)]:\(String(reflecting: type(of: error)))", metadata: metadata) self.cluster.tell(.inbound(.handshakeFailed(self.handshakeOffer.to, error))) context.fireErrorCaught(error) } @@ -113,7 +118,9 @@ final class ReceivingHandshakeHandler: ChannelInboundHandler, RemovableChannelHa // TODO: formalize wire format... let offer = try self.readHandshakeOffer(bytes: &bytes) - self.log.debug("Received handshake offer from: [\(offer.from)] with protocol version: [\(offer.version)]") + self.log.debug("Received handshake offer from: [\(offer.from)] with protocol version: [\(offer.version)]", metadata: [ + "handshake/channel": "\(context.channel)" + ]) let promise = context.eventLoop.makePromise(of: Wire.HandshakeResponse.self) self.cluster.tell(.inbound(.handshakeOffer(offer, channel: context.channel, replyInto: promise))) diff --git a/Sources/DistributedActors/Cluster/Transport/WireMessages.swift b/Sources/DistributedActors/Cluster/Transport/WireMessages.swift index 2db8a3923..f2e34c71d 100644 --- a/Sources/DistributedActors/Cluster/Transport/WireMessages.swift +++ b/Sources/DistributedActors/Cluster/Transport/WireMessages.swift @@ -60,11 +60,16 @@ internal enum Wire { internal let version: Version // TODO: Maybe offeringToSpeakAtVersion or something like that? + /// The node accepting the handshake. + /// + /// This will always be the "local" node where the accept is being made. internal let from: UniqueNode - /// In order to avoid confusion with from/to, we name the `origin` the node which an offer was sent "from", + /// In order to avoid confusion with from/to, we name the `origin` the node which an *offer* was sent from, /// and we now reply to this handshake to it. This value is carried so the origin can confirm it indeed was /// intended for it, and not a previous incarnation of a system on the same network address. + /// + /// This will always be the "remote" node, with regards to where the accept is created. internal let origin: UniqueNode init(version: Version, from: UniqueNode, origin: UniqueNode) { diff --git a/Sources/DistributedActors/String+Extensions.swift b/Sources/DistributedActors/String+Extensions.swift index e2ee5754a..d7cd82fab 100644 --- a/Sources/DistributedActors/String+Extensions.swift +++ b/Sources/DistributedActors/String+Extensions.swift @@ -57,7 +57,11 @@ internal extension String.StringInterpolation { public extension String.StringInterpolation { mutating func appendInterpolation(_ value: T?, orElse defaultValue: String) { - self.appendLiteral("[\(value.map { "\($0)" } ?? defaultValue)]") + self.appendLiteral("\(value.map { "\($0)" } ?? defaultValue)") + } + + mutating func appendInterpolation(optional value: T?) { + self.appendLiteral("\(value.map { "\($0)" } ?? "nil")") } } diff --git a/Sources/DistributedActorsBenchmarks/ActorPingPongBenchmarks.swift b/Sources/DistributedActorsBenchmarks/ActorPingPongBenchmarks.swift index 56696bcd3..737425be4 100644 --- a/Sources/DistributedActorsBenchmarks/ActorPingPongBenchmarks.swift +++ b/Sources/DistributedActorsBenchmarks/ActorPingPongBenchmarks.swift @@ -40,7 +40,7 @@ private let moreThanCoresActors = cores * 2 public let ActorPingPongBenchmarks: [BenchmarkInfo] = [ BenchmarkInfo( - name: "ActorPingPongBenchmarks.bench_actors_ping_pong(twoActors)", + name: "ActorPingPongBenchmarks.bench_actors_ping_pong(twoActors = 2)", runFunction: bench_actors_ping_pong(numActors: twoActors), tags: [.actor], setUpFunction: { @@ -51,7 +51,7 @@ public let ActorPingPongBenchmarks: [BenchmarkInfo] = [ tearDownFunction: tearDown ), BenchmarkInfo( - name: "ActorPingPongBenchmarks.bench_actors_ping_pong(lessThanCoresActors)", + name: "ActorPingPongBenchmarks.bench_actors_ping_pong(lessThanCoresActors = \(lessThanCoresActors))", runFunction: bench_actors_ping_pong(numActors: lessThanCoresActors), tags: [.actor], setUpFunction: { @@ -62,7 +62,7 @@ public let ActorPingPongBenchmarks: [BenchmarkInfo] = [ tearDownFunction: tearDown ), BenchmarkInfo( - name: "ActorPingPongBenchmarks.bench_actors_ping_pong(sameAsCoresActors)", + name: "ActorPingPongBenchmarks.bench_actors_ping_pong(sameAsCoresActors = \(sameAsCoresActors))", runFunction: bench_actors_ping_pong(numActors: sameAsCoresActors), tags: [.actor], setUpFunction: { @@ -73,7 +73,7 @@ public let ActorPingPongBenchmarks: [BenchmarkInfo] = [ tearDownFunction: tearDown ), BenchmarkInfo( - name: "ActorPingPongBenchmarks.bench_actors_ping_pong(moreThanCoresActors)", + name: "ActorPingPongBenchmarks.bench_actors_ping_pong(moreThanCoresActors = \(moreThanCoresActors))", runFunction: bench_actors_ping_pong(numActors: moreThanCoresActors), tags: [.actor], setUpFunction: { @@ -239,7 +239,6 @@ private func bench_actors_ping_pong(numActors: Int) -> (Int) -> Void { .startPingPong( messagesPerPair: numMessagesPerActorPair, numActors: numActors, - // dispatcher: "", // not used throughput: 50, shutdownTimeout: .seconds(30), replyTo: benchmarkLatchRef diff --git a/Sources/DistributedActorsBenchmarks/ActorRemotePingPongBenchmarks.swift b/Sources/DistributedActorsBenchmarks/ActorRemotePingPongBenchmarks.swift index ab3b9c60c..b837fcedf 100644 --- a/Sources/DistributedActorsBenchmarks/ActorRemotePingPongBenchmarks.swift +++ b/Sources/DistributedActorsBenchmarks/ActorRemotePingPongBenchmarks.swift @@ -26,12 +26,32 @@ import Glibc // MARK: Remote Ping Pong Benchmark -private let twoActors = 2 - public let ActorRemotePingPongBenchmarks: [BenchmarkInfo] = [ BenchmarkInfo( - name: "ActorRemotePingPongBenchmarks.bench_actors_remote_ping_pong(twoActors)", - runFunction: bench_actors_remote_ping_pong(numActors: twoActors), + name: "ActorRemotePingPongBenchmarks.bench_actors_remote_ping_pong(2)", + runFunction: bench_actors_remote_ping_pong(numActors: 2), + tags: [.actor], + setUpFunction: { + setUp { () in + supervisor = try! system.spawn("supervisor", supervisorBehavior()) + } + }, + tearDownFunction: tearDown + ), + BenchmarkInfo( + name: "ActorRemotePingPongBenchmarks.bench_actors_remote_ping_pong(8)", + runFunction: bench_actors_remote_ping_pong(numActors: 8), + tags: [.actor], + setUpFunction: { + setUp { () in + supervisor = try! system.spawn("supervisor", supervisorBehavior()) + } + }, + tearDownFunction: tearDown + ), + BenchmarkInfo( + name: "ActorRemotePingPongBenchmarks.bench_actors_remote_ping_pong(16)", + runFunction: bench_actors_remote_ping_pong(numActors: 16), tags: [.actor], setUpFunction: { setUp { () in diff --git a/Tests/ActorSingletonPluginTests/ActorSingletonPluginClusteredTests.swift b/Tests/ActorSingletonPluginTests/ActorSingletonPluginClusteredTests.swift index 0d62aae3e..9494306f0 100644 --- a/Tests/ActorSingletonPluginTests/ActorSingletonPluginClusteredTests.swift +++ b/Tests/ActorSingletonPluginTests/ActorSingletonPluginClusteredTests.swift @@ -173,17 +173,17 @@ final class ActorSingletonPluginClusteredTests: ClusteredNodesTestBase { try self.ensureNodes(.up, within: .seconds(10), nodes: first.cluster.node, second.cluster.node, third.cluster.node) let replyProbe1 = self.testKit(first).spawnTestProbe(expecting: String.self) - ref1.tell(.greet(name: "Charlie", _replyTo: replyProbe1.ref)) + ref1.tell(.greet(name: "Alice", _replyTo: replyProbe1.ref)) let replyProbe2 = self.testKit(second).spawnTestProbe(expecting: String.self) - ref2.tell(.greet(name: "Charlie", _replyTo: replyProbe2.ref)) + ref2.tell(.greet(name: "Bob", _replyTo: replyProbe2.ref)) let replyProbe3 = self.testKit(third).spawnTestProbe(expecting: String.self) ref3.tell(.greet(name: "Charlie", _replyTo: replyProbe3.ref)) // `first` has the lowest address so it should be the leader and singleton - try replyProbe1.expectMessage("Hello-1 Charlie!") - try replyProbe2.expectMessage("Hello-1 Charlie!") + try replyProbe1.expectMessage("Hello-1 Alice!") + try replyProbe2.expectMessage("Hello-1 Bob!") try replyProbe3.expectMessage("Hello-1 Charlie!") // Take down the leader @@ -200,8 +200,10 @@ final class ActorSingletonPluginClusteredTests: ClusteredNodesTestBase { try self.assertLeaderNode(on: third, is: nil) } + // ~~~~ racy ~~~~ + // No leader so singleton is not available, messages sent should be stashed - ref2.tell(.greet(name: "Charlie-2", _replyTo: replyProbe2.ref)) + ref2.tell(.greet(name: "Bob-2", _replyTo: replyProbe2.ref)) ref3.tell(.greet(name: "Charlie-3", _replyTo: replyProbe3.ref)) // `fourth` will become the new leader and singleton @@ -210,7 +212,7 @@ final class ActorSingletonPluginClusteredTests: ClusteredNodesTestBase { try self.ensureNodes(.up, on: second, within: .seconds(10), nodes: fourth.cluster.node, second.cluster.node, third.cluster.node) // The stashed messages get routed to new singleton running on `fourth` - try replyProbe2.expectMessage("Hello-4 Charlie-2!") + try replyProbe2.expectMessage("Hello-4 Bob-2!") try replyProbe3.expectMessage("Hello-4 Charlie-3!") } } diff --git a/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift b/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift index 631126248..0028e87be 100644 --- a/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift +++ b/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift @@ -30,6 +30,32 @@ final class ClusterAssociationTests: ClusteredNodesTestBase { try assertAssociated(remote, withExactly: local.cluster.node) } + func test_boundServer_shouldAcceptAssociate_raceFromBothNodes() throws { + let (local, remote) = self.setUpPair() + let n3 = self.setUpNode("node-3") + let n4 = self.setUpNode("node-4") + let n5 = self.setUpNode("node-5") + let n6 = self.setUpNode("node-6") + + local.cluster.join(node: remote.cluster.node.node) + remote.cluster.join(node: local.cluster.node.node) + + n3.cluster.join(node: local.cluster.node.node) + local.cluster.join(node: n3.cluster.node.node) + + n4.cluster.join(node: local.cluster.node.node) + local.cluster.join(node: n4.cluster.node.node) + + n5.cluster.join(node: local.cluster.node.node) + local.cluster.join(node: n5.cluster.node.node) + + n6.cluster.join(node: local.cluster.node.node) + local.cluster.join(node: n6.cluster.node.node) + + try assertAssociated(local, withAtLeast: remote.cluster.node) + try assertAssociated(remote, withAtLeast: local.cluster.node) + } + func test_handshake_shouldNotifyOnSuccess() throws { let (local, remote) = self.setUpPair() let p = self.testKit(local).spawnTestProbe(expecting: ClusterShell.HandshakeResult.self) @@ -285,9 +311,14 @@ final class ClusterAssociationTests: ClusteredNodesTestBase { try assertNotAssociated(system: first, node: second.cluster.node) try assertNotAssociated(system: second, node: first.cluster.node) - ref.tell("Hello!") // will be buffered until associated, and then delivered + // will be buffered until associated, and then delivered: + ref.tell("Hello 1") + ref.tell("Hello 2") + ref.tell("Hello 3") - try p2.expectMessage("Got:Hello!") + try p2.expectMessage("Got:Hello 1") + try p2.expectMessage("Got:Hello 2") + try p2.expectMessage("Got:Hello 3") try assertAssociated(first, withExactly: second.cluster.node) try assertAssociated(second, withExactly: first.cluster.node) diff --git a/Tests/DistributedActorsTests/Cluster/DowningStrategy/DowningClusteredTests.swift b/Tests/DistributedActorsTests/Cluster/DowningStrategy/DowningClusteredTests.swift index 407372121..72ca3a23c 100644 --- a/Tests/DistributedActorsTests/Cluster/DowningStrategy/DowningClusteredTests.swift +++ b/Tests/DistributedActorsTests/Cluster/DowningStrategy/DowningClusteredTests.swift @@ -41,75 +41,82 @@ final class DowningClusteredTests: ClusteredNodesTestBase { // ==== ---------------------------------------------------------------------------------------------------------------- // MARK: Downing - func shared_stoppingNode_shouldPropagateToOtherNodesAsDown(stopMethod: NodeStopMethod, stopNode: StopNodeSelection, _ modifySettings: ((inout ActorSystemSettings) -> Void)? = nil) throws { - let (first, second) = self.setUpPair(modifySettings) - let thirdNeverDownSystem = self.setUpNode("third", modifySettings) - - try self.joinNodes(node: first, with: second, ensureMembers: .up) - try self.joinNodes(node: thirdNeverDownSystem, with: second, ensureMembers: .up) - - let expectedDownSystem: ActorSystem - let otherNotDownPairSystem: ActorSystem - switch stopNode { - case .firstLeader: - expectedDownSystem = first - otherNotDownPairSystem = second - case .secondNonLeader: - expectedDownSystem = second - otherNotDownPairSystem = first - } + func shared_stoppingNode_shouldPropagateToOtherNodesAsDown( + stopMethod: NodeStopMethod, + stopNode: StopNodeSelection, + _ modifySettings: ((inout ActorSystemSettings) -> Void)? = nil + ) throws { + try shouldNotThrow { + + let (first, second) = self.setUpPair(modifySettings) + let thirdNeverDownSystem = self.setUpNode("third", modifySettings) + + try self.joinNodes(node: first, with: second, ensureMembers: .up) + try self.joinNodes(node: thirdNeverDownSystem, with: second, ensureMembers: .up) + + let expectedDownSystem: ActorSystem + let otherNotDownPairSystem: ActorSystem + switch stopNode { + case .firstLeader: + expectedDownSystem = first + otherNotDownPairSystem = second + case .secondNonLeader: + expectedDownSystem = second + otherNotDownPairSystem = first + } - let expectedDownNode = expectedDownSystem.cluster.node + let expectedDownNode = expectedDownSystem.cluster.node - // we start cluster event probes early, so they get the events one by one as they happen - let eventsProbeOther = self.testKit(otherNotDownPairSystem).spawnEventStreamTestProbe(subscribedTo: otherNotDownPairSystem.cluster.events) - let eventsProbeThird = self.testKit(thirdNeverDownSystem).spawnEventStreamTestProbe(subscribedTo: thirdNeverDownSystem.cluster.events) + // we start cluster event probes early, so they get the events one by one as they happen + let eventsProbeOther = self.testKit(otherNotDownPairSystem).spawnEventStreamTestProbe(subscribedTo: otherNotDownPairSystem.cluster.events) + let eventsProbeThird = self.testKit(thirdNeverDownSystem).spawnEventStreamTestProbe(subscribedTo: thirdNeverDownSystem.cluster.events) - pinfo("Expecting [\(expectedDownSystem)] to become [.down], method to stop the node [\(stopMethod)]") + pinfo("Expecting [\(expectedDownSystem)] to become [.down], method to stop the node [\(stopMethod)]") - // we cause the stop of the target node as expected - switch (stopMethod, stopNode) { - case (.leaveSelfNode, .firstLeader): first.cluster.leave() - case (.leaveSelfNode, .secondNonLeader): second.cluster.leave() + // we cause the stop of the target node as expected + switch (stopMethod, stopNode) { + case (.leaveSelfNode, .firstLeader): first.cluster.leave() + case (.leaveSelfNode, .secondNonLeader): second.cluster.leave() - case (.downSelf, .firstLeader): first.cluster.down(node: first.cluster.node.node) - case (.downSelf, .secondNonLeader): second.cluster.down(node: second.cluster.node.node) + case (.downSelf, .firstLeader): first.cluster.down(node: first.cluster.node.node) + case (.downSelf, .secondNonLeader): second.cluster.down(node: second.cluster.node.node) - case (.shutdownSelf, .firstLeader): first.shutdown() - case (.shutdownSelf, .secondNonLeader): second.shutdown() + case (.shutdownSelf, .firstLeader): first.shutdown() + case (.shutdownSelf, .secondNonLeader): second.shutdown() - case (.downFromOtherMember, .firstLeader): second.cluster.down(node: first.cluster.node.node) - case (.downFromOtherMember, .secondNonLeader): thirdNeverDownSystem.cluster.down(node: second.cluster.node.node) - } + case (.downFromOtherMember, .firstLeader): second.cluster.down(node: first.cluster.node.node) + case (.downFromOtherMember, .secondNonLeader): thirdNeverDownSystem.cluster.down(node: second.cluster.node.node) + } - func expectedDownMemberEventsFishing(on: ActorSystem) -> (Cluster.Event) -> ActorTestProbe.FishingDirective { - { event in - switch event { - case .membershipChange(let change) where change.node == expectedDownNode && change.isRemoval: - pinfo("MembershipChange on \(on.cluster.node.node): \(change)") - return .catchComplete(change) - case .membershipChange(let change) where change.node == expectedDownNode: - pinfo("MembershipChange on \(on.cluster.node.node): \(change)") - return .catchContinue(change) - case .reachabilityChange(let change) where change.member.node == expectedDownNode: - pnote("ReachabilityChange on \(otherNotDownPairSystem.cluster.node.node) = \(change)") - return .ignore - default: - // pnote("Event on \(otherNotDownPairSystem.cluster.node.node) = \(event)") - return .ignore + func expectedDownMemberEventsFishing(on: ActorSystem) -> (Cluster.Event) -> ActorTestProbe.FishingDirective { + { event in + switch event { + case .membershipChange(let change) where change.node == expectedDownNode && change.isRemoval: + pinfo("MembershipChange on \(on.cluster.node.node): \(change)") + return .catchComplete(change) + case .membershipChange(let change) where change.node == expectedDownNode: + pinfo("MembershipChange on \(on.cluster.node.node): \(change)") + return .catchContinue(change) + case .reachabilityChange(let change) where change.member.node == expectedDownNode: + pnote("ReachabilityChange on \(otherNotDownPairSystem.cluster.node.node) = \(change)") + return .ignore + default: + // pnote("Event on \(otherNotDownPairSystem.cluster.node.node) = \(event)") + return .ignore + } } } - } - // collect all events regarding the expectedDownNode's membership lifecycle - let eventsOnOther = try eventsProbeOther.fishFor(Cluster.MembershipChange.self, within: .seconds(20), expectedDownMemberEventsFishing(on: otherNotDownPairSystem)) - let eventsOnThird = try eventsProbeThird.fishFor(Cluster.MembershipChange.self, within: .seconds(20), expectedDownMemberEventsFishing(on: thirdNeverDownSystem)) + // collect all events regarding the expectedDownNode's membership lifecycle + let eventsOnOther = try eventsProbeOther.fishFor(Cluster.MembershipChange.self, within: .seconds(20), expectedDownMemberEventsFishing(on: otherNotDownPairSystem)) + let eventsOnThird = try eventsProbeThird.fishFor(Cluster.MembershipChange.self, within: .seconds(20), expectedDownMemberEventsFishing(on: thirdNeverDownSystem)) - eventsOnOther.shouldContain(where: { change in change.toStatus.isDown && (change.fromStatus == .joining || change.fromStatus == .up) }) - eventsOnOther.shouldContain(Cluster.MembershipChange(node: expectedDownNode, fromStatus: .down, toStatus: .removed)) + eventsOnOther.shouldContain(where: { change in change.toStatus.isDown && (change.fromStatus == .joining || change.fromStatus == .up) }) + eventsOnOther.shouldContain(Cluster.MembershipChange(node: expectedDownNode, fromStatus: .down, toStatus: .removed)) - eventsOnOther.shouldContain(where: { change in change.toStatus.isDown && (change.fromStatus == .joining || change.fromStatus == .up) }) - eventsOnThird.shouldContain(Cluster.MembershipChange(node: expectedDownNode, fromStatus: .down, toStatus: .removed)) + eventsOnOther.shouldContain(where: { change in change.toStatus.isDown && (change.fromStatus == .joining || change.fromStatus == .up) }) + eventsOnThird.shouldContain(Cluster.MembershipChange(node: expectedDownNode, fromStatus: .down, toStatus: .removed)) + } } // ==== ---------------------------------------------------------------------------------------------------------------- From d268d1539ced09ab2d8b678bc281a66d8afc2068 Mon Sep 17 00:00:00 2001 From: Konrad `ktoso` Malawski Date: Wed, 29 Apr 2020 18:40:35 +0900 Subject: [PATCH 08/18] fix the killing of the "good" connection as well --- Sources/DistributedActors/Behaviors.swift | 10 +-- .../Cluster/Association.swift | 19 +++-- .../Cluster/ClusterShell.swift | 73 +++++++++++-------- .../Cluster/ClusterShellState.swift | 25 ++++--- .../Cluster/HandshakeStateMachine.swift | 10 +-- .../Transport/TransportPipelines.swift | 47 ++++++------ .../DistributedActors/String+Extensions.swift | 9 +++ .../ActorSingletonPluginClusteredTests.swift | 6 +- .../Cluster/AssociationClusteredTests.swift | 6 +- .../DowningClusteredTests.swift | 7 +- 10 files changed, 119 insertions(+), 93 deletions(-) diff --git a/Sources/DistributedActors/Behaviors.swift b/Sources/DistributedActors/Behaviors.swift index 56bc1693f..8c33aab72 100644 --- a/Sources/DistributedActors/Behaviors.swift +++ b/Sources/DistributedActors/Behaviors.swift @@ -484,11 +484,11 @@ public extension Behavior { fatalError("Illegal to attempt to interpret message with .suspend behavior! Behavior should have been canonicalized. This is a bug, please open a ticket.", file: file, line: line) case .suspended: fatalError(""" - No message should ever be delivered to a .suspended behavior! - Message: \(message) - Actor: \(context) - This is a bug, please open a ticket. - """, file: file, line: line) + No message should ever be delivered to a .suspended behavior! + Message: \(message) + Actor: \(context) + This is a bug, please open a ticket. + """, file: file, line: line) } } diff --git a/Sources/DistributedActors/Cluster/Association.swift b/Sources/DistributedActors/Cluster/Association.swift index 1e0ea09d2..99b21aaff 100644 --- a/Sources/DistributedActors/Cluster/Association.swift +++ b/Sources/DistributedActors/Cluster/Association.swift @@ -62,12 +62,14 @@ final class Association: CustomStringConvertible { /// Complete the association and drain any pending message sends onto the channel. // TODO: This style can only ever work since we lock around the entirety of enqueueing messages and this setting; make it such that we don't need the lock eventually func completeAssociation(handshake: HandshakeStateMachine.CompletedState, over channel: Channel) { - assert(self.remoteNode == handshake.remoteNode, - """ - Complete association with wrong node was invoked. \ - Association, remote node: \(self.remoteNode); \ - Handshake, remote node: \(handshake.remoteNode) - """) + assert( + self.remoteNode == handshake.remoteNode, + """ + Complete association with wrong node was invoked. \ + Association, remote node: \(self.remoteNode); \ + Handshake, remote node: \(handshake.remoteNode) + """ + ) self.lock.withLockVoid { switch self.state { @@ -127,7 +129,7 @@ final class Association: CustomStringConvertible { self.lock.withLockVoid { switch self.state { case .associating: - () // TODO trigger when it completes + () // TODO: trigger when it completes case .associated: () // promise?.succeed(.accept(HandshakeAccept())) case .tombstone: @@ -156,13 +158,10 @@ extension Association { self.lock.withLockVoid { switch self.state { case .associating(let sendQueue): - pprint("SEND [ENQUEUE] = \(envelope.underlyingMessage) >>>> \(envelope.recipient)") sendQueue.enqueue(envelope) case .associated(let channel): - // pprint("SEND [SEND] = \(envelope.underlyingMessage) >>>> \(envelope.recipient)") channel.writeAndFlush(envelope, promise: promise) case .tombstone(let deadLetters): - pprint("SEND [DEAD] = \(envelope.underlyingMessage) >>>> \(envelope.recipient)") deadLetters.tell(.init(envelope.underlyingMessage, recipient: envelope.recipient)) } } diff --git a/Sources/DistributedActors/Cluster/ClusterShell.swift b/Sources/DistributedActors/Cluster/ClusterShell.swift index 49e6125ce..25676e1fe 100644 --- a/Sources/DistributedActors/Cluster/ClusterShell.swift +++ b/Sources/DistributedActors/Cluster/ClusterShell.swift @@ -619,7 +619,7 @@ extension ClusterShell { if let existingAssociation = self.getExistingAssociation(with: remoteNode) { state.log.debug("Association already allocated for remote: \(reflecting: remoteNode), existing association: [\(existingAssociation)]") switch existingAssociation.state { - case .associating(_): + case .associating: // whenComplete.futureResult.whenComplete { _ in // replyTo?.tell() // } @@ -714,9 +714,9 @@ extension ClusterShell { extension ClusterShell { /// Initial entry point for accepting a new connection; Potentially allocates new handshake state machine. internal func onHandshakeOffer( - _ context: ActorContext, _ state: ClusterShellState, - _ offer: Wire.HandshakeOffer, incomingChannel: Channel, - replyInto promise: EventLoopPromise + _ context: ActorContext, _ state: ClusterShellState, + _ offer: Wire.HandshakeOffer, incomingChannel: Channel, + replyInto promise: EventLoopPromise ) -> Behavior { var state = state @@ -728,7 +728,7 @@ extension ClusterShell { switch hsm.negotiate() { case .acceptAndAssociate(let completedHandshake): state.log.info("Accept association with \(reflecting: offer.from)!", metadata: [ - "handshake/channel": "\(incomingChannel)" + "handshake/channel": "\(incomingChannel)", ]) // accept handshake and store completed association @@ -810,7 +810,7 @@ extension ClusterShell { delay: delay ) case .giveUpOnHandshake: - if let hsmState = state.closeOutboundHandshakeChannel_TODOBETTERNAME(with: remoteNode) { + if let hsmState = state.closeOutboundHandshakeChannel(with: remoteNode) { self.notifyHandshakeFailure(state: hsmState, node: remoteNode, error: error) } } @@ -868,8 +868,6 @@ extension ClusterShell { self.recordMetrics(context.system.metrics, membership: state.membership) // 5) Finally, signal the handshake future that we've accepted, and become with ready state - print("handshakeCompleted.makeAccept() === \(handshakeCompleted.makeAccept())") - print(" accept === \(accept)") handshakeCompleted.whenCompleted?.succeed(.accept(handshakeCompleted.makeAccept())) return self.ready(state: state) } @@ -877,10 +875,10 @@ extension ClusterShell { /// An accept may imply that it replaced a previously associated member. /// If so, this method will .down it in the membership and terminate the previous instances association. private func handlePotentialAssociatedMemberReplacement( - directive: ClusterShellState.AssociatedDirective, - accept: Wire.HandshakeAccept, - context: ActorContext, - state: inout ClusterShellState + directive: ClusterShellState.AssociatedDirective, + accept: Wire.HandshakeAccept, + context: ActorContext, + state: inout ClusterShellState ) { if let replacedMember = directive.membershipChange.replaced { // the change was a replacement and thus we need to down the old member (same host:port as the new one), @@ -900,40 +898,55 @@ extension ClusterShell { } private func onHandshakeRejected(_ context: ActorContext, _ state: ClusterShellState, _ reject: Wire.HandshakeReject) -> Behavior { - var state = state - - state.log.error("Handshake was rejected by: [\(reject.from)], reason: [\(reject.reason)]") + // we MAY be seeing a handshake failure from a 2 nodes concurrently shaking hands on 2 connections, + // and we decided to tie-break and kill one of the connections. As such, the handshake COMPLETED successfully but + // on the other connection; and the terminated one may yield an error (e.g. truncation error during proto parsing etc), + // however that error is harmless - as we associated with the "other" right connection. + if let existingAssociation = self.getExistingAssociation(with: reject.from), + existingAssociation.isAssociated || existingAssociation.isTombstone { + state.log.debug( + "Handshake rejected by [\(reject.from)], however existing association with node exists. Could be that a concurrent handshake was failed on purpose.", + metadata: state.metadataForHandshakes(node: reject.from, error: nil) + ) + return .same + } // TODO: back off intensely, give up after some attempts? + // TODO: backoffs and retries schedule one here - if let hsmState = state.closeOutboundHandshakeChannel_TODOBETTERNAME(with: reject.from) { - self.notifyHandshakeFailure(state: hsmState, node: reject.from, error: HandshakeConnectionError(node: reject.from, message: reject.reason)) - } + state.log.warning( + "Handshake rejected by [\(reject.from)], reason: \(reject.reason)", + metadata: state.metadataForHandshakes(node: reject.from, error: nil) + ) - self.recordMetrics(context.system.metrics, membership: state.membership) - return self.ready(state: state) + return .same } private func onHandshakeFailed(_ context: ActorContext, _ state: ClusterShellState, with node: Node, error: Error) -> Behavior { -// var state = state - // we MAY be seeing a handshake failure from a 2 nodes concurrently shaking hands on 2 connections, // and we decided to tie-break and kill one of the connections. As such, the handshake COMPLETED successfully but // on the other connection; and the terminated one may yield an error (e.g. truncation error during proto parsing etc), // however that error is harmless - as we associated with the "other" right connection. if let existingAssociation = self.getExistingAssociation(with: node), - existingAssociation.isAssociated || existingAssociation.isTombstone { - state.log.trace("Handshake failed, however existing association with node exists. Could be that a concurrent handshake was failed on purpose.", - metadata: state.metadataForHandshakes(node: node, error: error)) + existingAssociation.isAssociated || existingAssociation.isTombstone { + state.log.debug( + "Handshake failed, however existing association with node exists. Could be that a concurrent handshake was failed on purpose.", + metadata: state.metadataForHandshakes(node: node, error: error) + ) return .same } -// state.log.error("Handshake error while connecting [\(node)]: \(error)", metadata: state.metadataForHandshakes(node: node, error: error)) -// if let hsmState = state.closeOutboundHandshakeChannel_TODOBETTERNAME(with: node) { -// self.notifyHandshakeFailure(state: hsmState, node: node, error: error) -// } + guard state.handshakeInProgress(with: node) != nil else { + state.log.debug("Received handshake failed notification, however handshake is not in progress, error: \(message: error)", metadata: [ + "handshake/node": "\(node)", + ]) + return .same + } - return self.ready(state: state) + // TODO: tweak logging some more, this is actually not scary in racy handshakes; so it may happen often + state.log.warning("Handshake error while connecting [\(node)]: \(error)", metadata: state.metadataForHandshakes(node: node, error: error)) + + return .same } private func onRestInPeace(_ context: ActorContext, _ state: ClusterShellState, intendedNode: UniqueNode, fromNode: UniqueNode) -> Behavior { diff --git a/Sources/DistributedActors/Cluster/ClusterShellState.swift b/Sources/DistributedActors/Cluster/ClusterShellState.swift index 4da4dff55..6ab7c3071 100644 --- a/Sources/DistributedActors/Cluster/ClusterShellState.swift +++ b/Sources/DistributedActors/Cluster/ClusterShellState.swift @@ -181,7 +181,7 @@ extension ClusterShellState { /// - Faults: when called in wrong state of an ongoing handshake /// - Returns: if present, the (now removed) handshake state that was aborted, hil otherwise. // THIS SIGNATURE IS SCARY; dont kill a Node, kill a specific channel (!) - mutating func closeOutboundHandshakeChannel_TODOBETTERNAME(with node: Node, file: String = #file, line: UInt = #line) -> HandshakeStateMachine.State? { + mutating func closeOutboundHandshakeChannel(with node: Node, file: String = #file, line: UInt = #line) -> HandshakeStateMachine.State? { guard let state = self._handshakes.removeValue(forKey: node) else { return nil } @@ -265,8 +265,7 @@ extension ClusterShellState { ]) if tieBreakWinner { - - if self.closeOutboundHandshakeChannel_TODOBETTERNAME(with: offer.from.node) != nil { + if self.closeOutboundHandshakeChannel(with: offer.from.node) != nil { self.log.debug( "Aborted handshake, as concurrently negotiating another one with same node already", metadata: [ @@ -442,12 +441,18 @@ extension ClusterShellState { ] } - func metadataForHandshakes(node: Node, error: Error) -> Logger.Metadata { - [ - "handshake/error": "\(error)", - "handshake/errorType": "\(String(reflecting: type(of: error as Any)))", - "handshake/peer": "\(node)", - "handshakes": "\(self.handshakes())" - ] + func metadataForHandshakes(node: Node, error err: Error?) -> Logger.Metadata { + var metadata: Logger.Metadata = + [ + "handshake/peer": "\(node)", + "handshakes": "\(self.handshakes())", + ] + + if let error = err { + metadata["handshake/error"] = "\(error)" + metadata["handshake/errorType"] = "\(String(reflecting: type(of: error as Any)))" + } + + return metadata } } diff --git a/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift b/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift index 2ddd3cc19..fc5f2a9e2 100644 --- a/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift +++ b/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift @@ -121,11 +121,11 @@ internal struct HandshakeStateMachine { var description: Swift.String { """ InitiatedState(\ - remoteNode: \(remoteNode), \ - localNode: \(localNode), \ - backoff: \(backoff), \ - whenCompleted: \(optional: whenCompleted), \ - channel: \(optional: channel)\ + remoteNode: \(self.remoteNode), \ + localNode: \(self.localNode), \ + backoff: \(self.backoff), \ + whenCompleted: \(optional: self.whenCompleted), \ + channel: \(optional: self.channel)\ ) """ } diff --git a/Sources/DistributedActors/Cluster/Transport/TransportPipelines.swift b/Sources/DistributedActors/Cluster/Transport/TransportPipelines.swift index 07632f1ba..2a6d9f197 100644 --- a/Sources/DistributedActors/Cluster/Transport/TransportPipelines.swift +++ b/Sources/DistributedActors/Cluster/Transport/TransportPipelines.swift @@ -50,10 +50,30 @@ private final class InitiatingHandshakeHandler: ChannelInboundHandler, Removable } func channelRead(context: ChannelHandlerContext, data: NIOAny) { - var bytes = self.unwrapInboundIn(data) + readHandshakeResponse(context: context, bytes: self.unwrapInboundIn(data)) + } + + private func initiateHandshake(context: ChannelHandlerContext) { + let proto = ProtoHandshakeOffer(self.handshakeOffer) + self.log.trace("Offering handshake [\(proto)]") + + do { + let bytes: ByteBuffer = try proto.serializedByteBuffer(allocator: context.channel.allocator) + // TODO: should we use the serialization infra ourselves here? I guess so... + + // FIXME: make the promise dance here + context.writeAndFlush(self.wrapOutboundOut(bytes), promise: nil) + } catch { + // TODO: change since serialization which can throw should be shipped of to a future + // ---- since now we blocked the actor basically with the serialization + context.fireErrorCaught(error) + } + } + private func readHandshakeResponse(context: ChannelHandlerContext, bytes: InboundIn) { + var bytes = bytes let metadata: Logger.Metadata = [ - "handshake/channel": "\(context.channel)" + "handshake/channel": "\(context.channel)", ] do { @@ -74,26 +94,7 @@ private final class InitiatingHandshakeHandler: ChannelInboundHandler, Removable } catch { self.log.debug("Handshake failure, error [\(error)]:\(String(reflecting: type(of: error)))", metadata: metadata) self.cluster.tell(.inbound(.handshakeFailed(self.handshakeOffer.to, error))) - context.fireErrorCaught(error) - } - } - - private func initiateHandshake(context: ChannelHandlerContext) { - let proto = ProtoHandshakeOffer(self.handshakeOffer) - self.log.trace("Offering handshake [\(proto)]") - - do { - // TODO: allow allocating into existing buffer - // FIXME: serialization SHOULD be on dedicated part... put it into ELF already? - let bytes: ByteBuffer = try proto.serializedByteBuffer(allocator: context.channel.allocator) - // TODO: should we use the serialization infra ourselves here? I guess so... - - // FIXME: make the promise dance here - context.writeAndFlush(self.wrapOutboundOut(bytes), promise: nil) - } catch { - // TODO: change since serialization which can throw should be shipped of to a future - // ---- since now we blocked the actor basically with the serialization - context.fireErrorCaught(error) + _ = context.close(mode: .all) } } } @@ -119,7 +120,7 @@ final class ReceivingHandshakeHandler: ChannelInboundHandler, RemovableChannelHa let offer = try self.readHandshakeOffer(bytes: &bytes) self.log.debug("Received handshake offer from: [\(offer.from)] with protocol version: [\(offer.version)]", metadata: [ - "handshake/channel": "\(context.channel)" + "handshake/channel": "\(context.channel)", ]) let promise = context.eventLoop.makePromise(of: Wire.HandshakeResponse.self) diff --git a/Sources/DistributedActors/String+Extensions.swift b/Sources/DistributedActors/String+Extensions.swift index d7cd82fab..17042037d 100644 --- a/Sources/DistributedActors/String+Extensions.swift +++ b/Sources/DistributedActors/String+Extensions.swift @@ -33,6 +33,15 @@ internal extension String.StringInterpolation { } } +// ==== ---------------------------------------------------------------------------------------------------------------- +// MARK: String Interpolation: Message printing [contents]:type which is useful for enums + +internal extension String.StringInterpolation { + mutating func appendInterpolation(message: Any) { + self.appendLiteral("[\(message)]:\(String(reflecting: type(of: message)))") + } +} + // ==== ---------------------------------------------------------------------------------------------------------------- // MARK: String Interpolation: reflecting: diff --git a/Tests/ActorSingletonPluginTests/ActorSingletonPluginClusteredTests.swift b/Tests/ActorSingletonPluginTests/ActorSingletonPluginClusteredTests.swift index 9494306f0..5b4aeed43 100644 --- a/Tests/ActorSingletonPluginTests/ActorSingletonPluginClusteredTests.swift +++ b/Tests/ActorSingletonPluginTests/ActorSingletonPluginClusteredTests.swift @@ -54,7 +54,7 @@ final class ActorSingletonPluginClusteredTests: ClusteredNodesTestBase { third.cluster.join(node: second.cluster.node.node) // `first` will be the leader (lowest address) and runs the singleton - try self.ensureNodes(.up, within: .seconds(10), nodes: first.cluster.node, second.cluster.node, third.cluster.node) + try self.ensureNodes(.up, within: .seconds(15), nodes: first.cluster.node, second.cluster.node, third.cluster.node) let replyProbe1 = self.testKit(first).spawnTestProbe(expecting: String.self) ref1.tell(.greet(name: "Charlie", _replyTo: replyProbe1.ref)) @@ -119,7 +119,7 @@ final class ActorSingletonPluginClusteredTests: ClusteredNodesTestBase { third.cluster.join(node: second.cluster.node.node) // `first` becomes the leader (lowest address) and runs the singleton - try self.ensureNodes(.up, within: .seconds(10), nodes: first.cluster.node, second.cluster.node, third.cluster.node) + try self.ensureNodes(.up, within: .seconds(15), nodes: first.cluster.node, second.cluster.node, third.cluster.node) try replyProbe1.expectMessage("Hello-1 Charlie-1!") try replyProbe2.expectMessage("Hello-1 Charlie-2!") @@ -170,7 +170,7 @@ final class ActorSingletonPluginClusteredTests: ClusteredNodesTestBase { first.cluster.join(node: second.cluster.node.node) third.cluster.join(node: second.cluster.node.node) - try self.ensureNodes(.up, within: .seconds(10), nodes: first.cluster.node, second.cluster.node, third.cluster.node) + try self.ensureNodes(.up, within: .seconds(15), nodes: first.cluster.node, second.cluster.node, third.cluster.node) let replyProbe1 = self.testKit(first).spawnTestProbe(expecting: String.self) ref1.tell(.greet(name: "Alice", _replyTo: replyProbe1.ref)) diff --git a/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift b/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift index 0028e87be..80c704cf2 100644 --- a/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift +++ b/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift @@ -312,9 +312,9 @@ final class ClusterAssociationTests: ClusteredNodesTestBase { try assertNotAssociated(system: second, node: first.cluster.node) // will be buffered until associated, and then delivered: - ref.tell("Hello 1") - ref.tell("Hello 2") - ref.tell("Hello 3") + ref.tell("Hello 1") + ref.tell("Hello 2") + ref.tell("Hello 3") try p2.expectMessage("Got:Hello 1") try p2.expectMessage("Got:Hello 2") diff --git a/Tests/DistributedActorsTests/Cluster/DowningStrategy/DowningClusteredTests.swift b/Tests/DistributedActorsTests/Cluster/DowningStrategy/DowningClusteredTests.swift index 72ca3a23c..61d5b999a 100644 --- a/Tests/DistributedActorsTests/Cluster/DowningStrategy/DowningClusteredTests.swift +++ b/Tests/DistributedActorsTests/Cluster/DowningStrategy/DowningClusteredTests.swift @@ -42,12 +42,11 @@ final class DowningClusteredTests: ClusteredNodesTestBase { // MARK: Downing func shared_stoppingNode_shouldPropagateToOtherNodesAsDown( - stopMethod: NodeStopMethod, - stopNode: StopNodeSelection, - _ modifySettings: ((inout ActorSystemSettings) -> Void)? = nil + stopMethod: NodeStopMethod, + stopNode: StopNodeSelection, + _ modifySettings: ((inout ActorSystemSettings) -> Void)? = nil ) throws { try shouldNotThrow { - let (first, second) = self.setUpPair(modifySettings) let thirdNeverDownSystem = self.setUpNode("third", modifySettings) From 674b1f2ba692f0ac49921ad23f51e15f93844a16 Mon Sep 17 00:00:00 2001 From: Konrad `ktoso` Malawski Date: Wed, 29 Apr 2020 19:00:23 +0900 Subject: [PATCH 09/18] =test,fix default address uses 127.0.0.1 fix test expectation --- .../CRDT/Protobuf/CRDT+SerializationTests.swift | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Tests/DistributedActorsTests/CRDT/Protobuf/CRDT+SerializationTests.swift b/Tests/DistributedActorsTests/CRDT/Protobuf/CRDT+SerializationTests.swift index 2d7b197f3..59e6c6e63 100644 --- a/Tests/DistributedActorsTests/CRDT/Protobuf/CRDT+SerializationTests.swift +++ b/Tests/DistributedActorsTests/CRDT/Protobuf/CRDT+SerializationTests.swift @@ -419,10 +419,10 @@ final class CRDTSerializationTests: ActorSystemTestBase { let serialized = try system.serialization.serialize(register) let deserialized = try system.serialization.deserialize(as: CRDT.LWWRegister.self, from: serialized) - "\(deserialized.replicaID)".shouldContain("actor:sact://CRDTSerializationTests@localhost:9001/user/alpha") + "\(deserialized.replicaID)".shouldContain("actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/alpha") deserialized.initialValue.shouldEqual(6) deserialized.value.shouldEqual(8) - "\(deserialized.updatedBy)".shouldContain("actor:sact://CRDTSerializationTests@localhost:9001/user/alpha") + "\(deserialized.updatedBy)".shouldContain("actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/alpha") // `TimeInterval` is `Double` XCTAssertEqual(deserialized.clock.timestamp.timeIntervalSince1970, clock.timestamp.timeIntervalSince1970, accuracy: 1) From 8e1454e489497c8d7f8ba1dba2d940a4a6126e4f Mon Sep 17 00:00:00 2001 From: Konrad `ktoso` Malawski Date: Wed, 29 Apr 2020 23:32:28 +0900 Subject: [PATCH 10/18] =test better naming in handshake messages solve the handshake truncation issue; ordering of flushing and sending the handshake we solve this by strongly ordering that flushing messages only happens after we write the handshake reply; as well as signaling the replyTo of the handshake only once we've swapped the association to ready --- Protos/WireProtocol.proto | 16 +- .../Cluster/Association.swift | 57 +++++-- .../Cluster/ClusterShell+LeaderActions.swift | 2 +- .../Cluster/ClusterShell.swift | 160 +++++++++++------- .../Cluster/ClusterShellState.swift | 55 +++--- .../Cluster/HandshakeStateMachine.swift | 80 +++++---- .../Transport/TransportPipelines.swift | 108 ++++++------ .../Cluster/Transport/WireMessages.swift | 33 ++-- .../Protobuf/WireProtocol+Serialization.swift | 60 +++---- .../Protobuf/WireProtocol.pb.swift | 158 +++++++++-------- .../Cluster/ClusteredNodesTestBase.swift | 2 +- .../ActorSingletonPluginClusteredTests.swift | 88 ++++++---- .../Cluster/AssociationClusteredTests.swift | 36 ++-- .../Cluster/ClusterLeaderActionsTests.swift | 6 +- .../Cluster/MembershipGossipTests.swift | 4 - .../Cluster/ProtobufRoundTripTests.swift | 2 +- .../RemotingHandshakeStateMachineTests.swift | 27 +-- .../ParentChildActorTests.swift | 2 - 18 files changed, 505 insertions(+), 391 deletions(-) diff --git a/Protos/WireProtocol.proto b/Protos/WireProtocol.proto index ae6fd6a2c..c9c3d6fad 100644 --- a/Protos/WireProtocol.proto +++ b/Protos/WireProtocol.proto @@ -25,8 +25,8 @@ import "Serialization/Serialization.proto"; message HandshakeOffer { ProtocolVersion version = 1; - UniqueNode from = 2; - Node to = 3; + UniqueNode originNode = 2; + Node targetNode = 3; // In the future we may want to add additional information // about certain capabilities here. E.g. when a node supports // faster transport like InfiniBand and the likes, so we can @@ -43,16 +43,14 @@ message HandshakeResponse { message HandshakeAccept { ProtocolVersion version = 1; - UniqueNode origin = 2; - UniqueNode from = 3; + UniqueNode originNode = 2; + UniqueNode targetNode = 3; } message HandshakeReject { - ProtocolVersion version = 1; - UniqueNode origin = 2; - // In the reject case this is an `Node` instead of a `UniqueNode`, - // to explicitly prevent this from forming an association. - Node from = 3; + ProtocolVersion version = 1; + UniqueNode originNode = 2; + UniqueNode targetNode = 3; string reason = 4; } diff --git a/Sources/DistributedActors/Cluster/Association.swift b/Sources/DistributedActors/Cluster/Association.swift index 99b21aaff..c814c4376 100644 --- a/Sources/DistributedActors/Cluster/Association.swift +++ b/Sources/DistributedActors/Cluster/Association.swift @@ -41,6 +41,9 @@ final class Association: CustomStringConvertible { // We'd prefer to have a lock-less way to implement this and we can achieve it but it's a pain to implement so will be done in a separate step. var state: State + /// Tasks to be executed when transitioning to completed/terminated state. + private var completionTasks: [() -> Void] + enum State { case associating(queue: MPSCLinkedQueue) case associated(channel: Channel) // TODO: ActorTransport.Node/Peer/Target ??? @@ -57,6 +60,7 @@ final class Association: CustomStringConvertible { self.remoteNode = remoteNode self.lock = Lock() self.state = .associating(queue: .init()) + self.completionTasks = [] } /// Complete the association and drain any pending message sends onto the channel. @@ -74,10 +78,7 @@ final class Association: CustomStringConvertible { self.lock.withLockVoid { switch self.state { case .associating(let sendQueue): - // 1) store associated channel - self.state = .associated(channel: channel) - - // 2) we need to flush all the queued up messages + // 1) we need to flush all the queued up messages // - yes, we need to flush while holding the lock... it's an annoyance in this lock based design // but it ensures that once we've flushed, all other messages will be sent in the proper order "after" // the previously enqueued ones; A lockless design would not be able to get rid of the queue AFAIR, @@ -85,8 +86,14 @@ final class Association: CustomStringConvertible { _ = channel.writeAndFlush(envelope) } + // 2) execute any pending tasks and clear them + self.runCompletionTasks() + + // 3) store associated channel + self.state = .associated(channel: channel) + case .associated: - _ = channel.close() + _ = channel.close() // TODO: throw instead of accepting a "double complete"? case .tombstone: _ = channel.close() @@ -94,6 +101,24 @@ final class Association: CustomStringConvertible { } } + /// Sometimes, while the association is still associating, we may need to enqueue tasks to be performed after it has completed. + /// This function ensures that once the `completeAssociation` is invoked and all messages flushed, the enqueued tasks will be executed (in same order as submitted). + /// + /// If the association is already associated or terminated, the task is executed immediately + // Implementation note: + // We can't use futures here because we may not have an event loop to hand it off; + // With enough weaving/exposing things inside ActorPersonality we could make it so, but not having to do so feel somewhat cleaner... + func enqueueCompletionTask(_ task: @escaping () -> Void) { + self.lock.withLockVoid { + switch self.state { + case .associating: + self.completionTasks.append(task) + default: + task() + } + } + } + /// Terminate the association and store a tombstone in it. /// /// If any messages were still queued up in it, or if it was hosting a channel these get drained / closed, @@ -107,6 +132,7 @@ final class Association: CustomStringConvertible { while let envelope = sendQueue.dequeue() { system.deadLetters.tell(.init(envelope.underlyingMessage, recipient: envelope.recipient)) } + self.runCompletionTasks() // in case someone stored a reference to this association in a ref, we swap it into a dead letter sink self.state = .tombstone(system.deadLetters) case .associated(let channel): @@ -121,21 +147,16 @@ final class Association: CustomStringConvertible { return Association.Tombstone(self.remoteNode, settings: system.settings.cluster) } - var description: String { - "AssociatedState(\(self.state), selfNode: \(self.selfNode), remoteNode: \(self.remoteNode))" + /// Runs and clears completion tasks. + private func runCompletionTasks() { + for task in self.completionTasks { + task() + } + self.completionTasks = [] } - func whenComplete(_ promise: EventLoopPromise?) { - self.lock.withLockVoid { - switch self.state { - case .associating: - () // TODO: trigger when it completes - case .associated: - () // promise?.succeed(.accept(HandshakeAccept())) - case .tombstone: - promise?.fail(HandshakeError.targetAlreadyTombstone(selfNode: self.selfNode, remoteNode: self.remoteNode)) - } - } + var description: String { + "AssociatedState(\(self.state), selfNode: \(self.selfNode), remoteNode: \(self.remoteNode))" } } diff --git a/Sources/DistributedActors/Cluster/ClusterShell+LeaderActions.swift b/Sources/DistributedActors/Cluster/ClusterShell+LeaderActions.swift index 83e4ae36a..3d4b389d7 100644 --- a/Sources/DistributedActors/Cluster/ClusterShell+LeaderActions.swift +++ b/Sources/DistributedActors/Cluster/ClusterShell+LeaderActions.swift @@ -22,7 +22,7 @@ import NIO extension ClusterShellState { /// If, and only if, the current node is a leader it performs a set of tasks, such as moving nodes to `.up` etc. func collectLeaderActions() -> [LeaderAction] { - guard self.membership.isLeader(self.myselfNode) else { + guard self.membership.isLeader(self.localNode) else { return [] // since we are not the leader, we perform no tasks } diff --git a/Sources/DistributedActors/Cluster/ClusterShell.swift b/Sources/DistributedActors/Cluster/ClusterShell.swift index 25676e1fe..c7fb8578b 100644 --- a/Sources/DistributedActors/Cluster/ClusterShell.swift +++ b/Sources/DistributedActors/Cluster/ClusterShell.swift @@ -49,7 +49,7 @@ internal class ClusterShell { /// - Protected by: `_associationsLock` private var _associationTombstones: [UniqueNode: Association.Tombstone] - internal func getExistingAssociation(with node: Node) -> Association? { + internal func getAnyExistingAssociation(with node: Node) -> Association? { self._associationsLock.withLock { // TODO: a bit terrible; perhaps we should key be Node and then confirm by UniqueNode? // this used to be separated in the State keeping them by Node and here we kept by unique though that caused other challenges @@ -59,6 +59,12 @@ internal class ClusterShell { } } + internal func getSpecificExistingAssociation(with node: UniqueNode) -> Association? { + self._associationsLock.withLock { + self._associations[node] + } + } + /// Get an existing association or ensure that a new one shall be stored and joining kicked off if the target node was not known yet. /// Safe to concurrently access by privileged internals directly internal func getEnsureAssociation(with node: UniqueNode) -> StoredAssociationState { @@ -89,7 +95,7 @@ internal class ClusterShell { /// To be invoked by cluster shell whenever handshake is accepted, creating a completed association. /// Causes messages to be flushed onto the new associated channel. - private func storeCompleteAssociation(_ associated: ClusterShellState.AssociatedDirective) { + private func completeAssociation(_ associated: ClusterShellState.AssociatedDirective) { self._associationsLock.withLockVoid { let association = self._associations[associated.handshake.remoteNode] ?? Association(selfNode: associated.handshake.localNode, remoteNode: associated.handshake.remoteNode) @@ -302,7 +308,7 @@ internal class ClusterShell { } internal enum InboundMessage { - case handshakeOffer(Wire.HandshakeOffer, channel: Channel, replyInto: EventLoopPromise) + case handshakeOffer(Wire.HandshakeOffer, channel: Channel, handshakeReplyTo: EventLoopPromise) case handshakeAccepted(Wire.HandshakeAccept, channel: Channel) case handshakeRejected(Wire.HandshakeReject) case handshakeFailed(Node, Error) // TODO: remove? @@ -320,8 +326,8 @@ internal class ClusterShell { case failure(HandshakeConnectionError) } - struct HandshakeConnectionError: Error, Equatable { // TODO: merge with HandshakeError? - let node: Node + struct HandshakeConnectionError: Error, Equatable { + let node: Node // TODO: allow carrying UniqueNode let message: String } @@ -403,8 +409,8 @@ extension ClusterShell { ) // loop through "self" cluster shell, which in result causes notifying all subscribers about cluster membership change - var firstGossip = Cluster.Gossip(ownerNode: state.myselfNode) - _ = firstGossip.membership.join(state.myselfNode) // change will be put into effect by receiving the "self gossip" + var firstGossip = Cluster.Gossip(ownerNode: state.localNode) + _ = firstGossip.membership.join(state.localNode) // change will be put into effect by receiving the "self gossip" context.system.cluster.updateMembershipSnapshot(state.membership) firstGossip.incrementOwnerVersion() context.myself.tell(.gossipFromGossiper(firstGossip)) @@ -472,15 +478,15 @@ extension ClusterShell { func receiveInbound(_ context: ActorContext, message: InboundMessage) throws -> Behavior { switch message { case .handshakeOffer(let offer, let channel, let promise): - self.tracelog(context, .receiveUnique(from: offer.from), message: offer) - return self.onHandshakeOffer(context, state, offer, incomingChannel: channel, replyInto: promise) + self.tracelog(context, .receiveUnique(from: offer.originNode), message: offer) + return self.onHandshakeOffer(context, state, offer, inboundChannel: channel, replyInto: promise) case .handshakeAccepted(let accepted, let channel): - self.tracelog(context, .receiveUnique(from: accepted.from), message: accepted) + self.tracelog(context, .receiveUnique(from: accepted.targetNode), message: accepted) return self.onHandshakeAccepted(context, state, accepted, channel: channel) case .handshakeRejected(let rejected): - self.tracelog(context, .receive(from: rejected.from), message: rejected) + self.tracelog(context, .receiveUnique(from: rejected.targetNode), message: rejected) return self.onHandshakeRejected(context, state, rejected) case .handshakeFailed(let fromNode, let error): @@ -495,7 +501,7 @@ extension ClusterShell { /// Allows processing in one spot, all membership changes which we may have emitted in other places, due to joining, downing etc. func receiveChangeMembershipRequest(_ context: ActorContext, event: Cluster.Event) -> Behavior { - self.tracelog(context, .receive(from: state.myselfNode.node), message: event) + self.tracelog(context, .receive(from: state.localNode.node), message: event) var state = state let changeDirective = state.applyClusterEvent(event) @@ -583,7 +589,7 @@ extension ClusterShell { guard change.toStatus < .down else { return } - guard change.member.node != state.myselfNode else { + guard change.member.node != state.localNode else { return } // TODO: make it cleaner? though we decided to go with manual peer management as the ClusterShell owns it, hm @@ -609,21 +615,21 @@ extension ClusterShell { internal func beginHandshake(_ context: ActorContext, _ state: ClusterShellState, with remoteNode: Node, replyTo: ActorRef?) -> Behavior { var state = state - guard remoteNode != state.myselfNode.node else { + guard remoteNode != state.localNode.node else { state.log.debug("Ignoring attempt to handshake with myself; Could have been issued as confused attempt to handshake as induced by discovery via gossip?") replyTo?.tell(.failure(.init(node: remoteNode, message: "Would have attempted handshake with self node, aborted handshake."))) return .same } // if an association exists for any UniqueNode that this Node represents, we can use this and abort the handshake dance here - if let existingAssociation = self.getExistingAssociation(with: remoteNode) { + if let existingAssociation = self.getAnyExistingAssociation(with: remoteNode) { state.log.debug("Association already allocated for remote: \(reflecting: remoteNode), existing association: [\(existingAssociation)]") switch existingAssociation.state { case .associating: -// whenComplete.futureResult.whenComplete { _ in -// replyTo?.tell() -// } - () // continue, we may be the first beginHandshake (as associations may be ensured outside of actor context) + // continue, we may be the first beginHandshake (as associations may be ensured outside of actor context) + existingAssociation.enqueueCompletionTask { + replyTo?.tell(.success(state.localNode)) + } case .associated: // return , we've been successful already replyTo?.tell(.success(existingAssociation.remoteNode)) @@ -643,15 +649,15 @@ extension ClusterShell { whenHandshakeComplete.futureResult.whenComplete { result in switch result { case .success(.accept(let accept)): - replyTo?.tell(.success(accept.origin)) + replyTo?.tell(.success(accept.originNode)) case .success(.reject(let reject)): replyTo?.tell(.failure(HandshakeConnectionError(node: remoteNode, message: reject.reason))) case .failure(let error): - replyTo?.tell(HandshakeResult.failure(.init(node: remoteNode, message: "\(error)"))) + replyTo?.tell(HandshakeResult.failure(HandshakeConnectionError(node: remoteNode, message: "\(error)"))) } } - let handshakeState = state.registerHandshake(with: remoteNode, whenCompleted: whenHandshakeComplete) + let handshakeState = state.beginHandshake(with: remoteNode, whenCompleted: whenHandshakeComplete) // we MUST register the intention of shaking hands with remoteAddress before obtaining the connection, // in order to let the fsm handle any retry decisions in face of connection failures et al. @@ -713,31 +719,32 @@ extension ClusterShell { extension ClusterShell { /// Initial entry point for accepting a new connection; Potentially allocates new handshake state machine. + /// - parameter inboundChannel: the inbound connection channel that the other node has opened and is offering its handshake on, + /// (as opposed to the channel which we may have opened when we first extended a handshake to that node which would be stored in `state`) internal func onHandshakeOffer( _ context: ActorContext, _ state: ClusterShellState, - _ offer: Wire.HandshakeOffer, incomingChannel: Channel, + _ offer: Wire.HandshakeOffer, inboundChannel: Channel, replyInto promise: EventLoopPromise ) -> Behavior { var state = state - state.log.notice("HANDLING ON HANDSHAKE OFFER \(offer) >>> \(incomingChannel)!") - - switch state.onIncomingHandshakeOffer(offer: offer, incomingChannel: incomingChannel) { + switch state.onIncomingHandshakeOffer(offer: offer, incomingChannel: inboundChannel) { case .negotiateIncoming(let hsm): // handshake is allowed to proceed switch hsm.negotiate() { case .acceptAndAssociate(let completedHandshake): - state.log.info("Accept association with \(reflecting: offer.from)!", metadata: [ - "handshake/channel": "\(incomingChannel)", + state.log.info("Accept handshake with \(reflecting: offer.originNode)!", metadata: [ + "handshake/channel": "\(inboundChannel)", ]) // accept handshake and store completed association - let directive = state.associate(context.system, completedHandshake, channel: incomingChannel) - self.storeCompleteAssociation(directive) + let directive = state.completeHandshakeAssociate(self, completedHandshake, channel: inboundChannel) // send accept to other node - let accept = completedHandshake.makeAccept() - self.tracelog(context, .send(to: offer.from.node), message: accept) + let accept = completedHandshake.makeAccept(whenHandshakeReplySent: { () in + self.completeAssociation(directive) + }) + self.tracelog(context, .send(to: offer.originNode.node), message: accept) promise.succeed(.accept(accept)) // This association may mean that we've "replaced" a previously known node of the same host:port, @@ -760,27 +767,31 @@ extension ClusterShell { return self.ready(state: state) case .rejectHandshake(let rejectedHandshake): - state.log.warning("Rejecting handshake from \(offer.from), error: [\(rejectedHandshake.error)]:\(type(of: rejectedHandshake.error))") + state.log.warning("Rejecting handshake from \(offer.originNode), error: [\(rejectedHandshake.error)]:\(type(of: rejectedHandshake.error))") // note that we should NOT abort the channel here since we still want to send back the rejection. - let reject: Wire.HandshakeReject = rejectedHandshake.makeReject() - self.tracelog(context, .send(to: offer.from.node), message: reject) + let reject: Wire.HandshakeReject = rejectedHandshake.makeReject(whenHandshakeReplySent: { () in + self.terminateAssociation(context.system, state: &state, rejectedHandshake.remoteNode) + }) + self.tracelog(context, .send(to: offer.originNode.node), message: reject) promise.succeed(.reject(reject)) return self.ready(state: state) } + case .abortIncomingDueToConcurrentHandshake: // concurrent handshake and we should abort let error = HandshakeConnectionError( - node: offer.from.node, + node: offer.originNode.node, message: """ - Terminating this connection, as there is a concurrently established connection with same host [\(offer.from)] \ + Terminating this connection, as there is a concurrently established connection with same host [\(offer.originNode)] \ which will be used to complete the handshake. """ ) - state.closeHandshakeChannel(offer: offer, channel: incomingChannel) promise.fail(error) + + state.closeHandshakeChannel(offer: offer, channel: inboundChannel) return .same } } @@ -831,28 +842,34 @@ extension ClusterShell { // MARK: Incoming Handshake Replies extension ClusterShell { - private func onHandshakeAccepted(_ context: ActorContext, _ state: ClusterShellState, _ accept: Wire.HandshakeAccept, channel: Channel) -> Behavior { + private func onHandshakeAccepted(_ context: ActorContext, _ state: ClusterShellState, _ inboundAccept: Wire.HandshakeAccept, channel: Channel) -> Behavior { var state = state // local copy for mutation - guard let handshakeCompleted = state.incomingHandshakeAccept(accept) else { - if self._associatedNodes().contains(accept.from) { + state.log.info("Accept association with \(reflecting: inboundAccept.targetNode)!", metadata: [ + "handshake/localNode": "\(inboundAccept.originNode)", + "handshake/remoteNode": "\(inboundAccept.targetNode)", + "handshake/channel": "\(channel)", + ]) + + guard let handshakeCompleted = state.incomingHandshakeAccept(inboundAccept) else { + if self._associatedNodes().contains(inboundAccept.targetNode) { // this seems to be a re-delivered accept, we already accepted association with this node. return .same } else { - state.log.error("Unexpected handshake accept received: [\(accept)]. No handshake was in progress with \(accept.from)") + state.log.error("Unexpected handshake accept received: [\(inboundAccept)]. No handshake was in progress with \(inboundAccept.targetNode)") return .same } } - // 1) Associate the new node - let directive = state.associate(context.system, handshakeCompleted, channel: channel) + // 1) Complete the handshake (in the cluster state) + let directive = state.completeHandshakeAssociate(self, handshakeCompleted, channel: channel) // 1.1) This association may mean that we've "replaced" a previously known node of the same host:port, // In case of such replacement we must down and terminate the association of the previous node. - self.handlePotentialAssociatedMemberReplacement(directive: directive, accept: accept, context: context, state: &state) + self.handlePotentialAssociatedMemberReplacement(directive: directive, accept: inboundAccept, context: context, state: &state) // 2) Store the (now completed) association first, as it may be immediately used by remote ActorRefs attempting to send to the remoteNode - self.storeCompleteAssociation(directive) // we MUST store it outside the state + self.completeAssociation(directive) state.log.debug("Associated with: \(reflecting: handshakeCompleted.remoteNode); Membership change: \(directive.membershipChange), resulting in: \(state.membership)") // 3) publish any cluster events this association caused. @@ -868,7 +885,10 @@ extension ClusterShell { self.recordMetrics(context.system.metrics, membership: state.membership) // 5) Finally, signal the handshake future that we've accepted, and become with ready state - handshakeCompleted.whenCompleted?.succeed(.accept(handshakeCompleted.makeAccept())) + let accept: Wire.HandshakeAccept = handshakeCompleted.makeAccept(whenHandshakeReplySent: { () in + self.completeAssociation(directive) + }) + handshakeCompleted.whenCompleted?.succeed(.accept(accept)) return self.ready(state: state) } @@ -884,7 +904,7 @@ extension ClusterShell { // the change was a replacement and thus we need to down the old member (same host:port as the new one), // and terminate its association. - state.log.info("Accepted handshake from [\(accept.from)] which replaces the previously known: \(reflecting: replacedMember).") + state.log.info("Accepted handshake from [\(accept.targetNode)] which replaces the previously known: \(reflecting: replacedMember).") // We MUST be careful to first terminate the association and then store the new one in 2) self.terminateAssociation(context.system, state: &state, replacedMember.node) @@ -902,23 +922,41 @@ extension ClusterShell { // and we decided to tie-break and kill one of the connections. As such, the handshake COMPLETED successfully but // on the other connection; and the terminated one may yield an error (e.g. truncation error during proto parsing etc), // however that error is harmless - as we associated with the "other" right connection. - if let existingAssociation = self.getExistingAssociation(with: reject.from), + if let existingAssociation = self.getSpecificExistingAssociation(with: reject.targetNode), + existingAssociation.isAssociating { + state.log.warning( + "Handshake rejected by [\(reject.targetNode)], it was associating and is now tombstoned", + metadata: state.metadataForHandshakes(uniqueNode: reject.targetNode, error: nil) + ) + var state = state + self.terminateAssociation(context.system, state: &state, reject.targetNode) + return self.ready(state: state) + } + + if let existingAssociation = self.getAnyExistingAssociation(with: reject.targetNode.node), existingAssociation.isAssociated || existingAssociation.isTombstone { state.log.debug( - "Handshake rejected by [\(reject.from)], however existing association with node exists. Could be that a concurrent handshake was failed on purpose.", - metadata: state.metadataForHandshakes(node: reject.from, error: nil) + "Handshake rejected by [\(reject.targetNode)], however existing association with node exists. Could be that a concurrent handshake was failed on purpose.", + metadata: state.metadataForHandshakes(uniqueNode: reject.targetNode, error: nil) ) return .same } - // TODO: back off intensely, give up after some attempts? - // TODO: backoffs and retries schedule one here - state.log.warning( - "Handshake rejected by [\(reject.from)], reason: \(reject.reason)", - metadata: state.metadataForHandshakes(node: reject.from, error: nil) + "Handshake rejected by [\(reject.targetNode)], reason: \(reject.reason)", + metadata: state.metadataForHandshakes(uniqueNode: reject.targetNode, error: nil) ) + // TODO: back off and retry, give up after some attempts; count which failure this was, update the handshake state + + if let handshakeState = state.handshakeInProgress(with: reject.targetNode.node) { + self.notifyHandshakeFailure( + state: handshakeState, + node: reject.targetNode.node, + error: HandshakeError.targetRejectedHandshake(selfNode: state.localNode, remoteNode: reject.targetNode, message: reject.reason) + ) + } // else, seems that node successfully associated and this may be for a previous connection + return .same } @@ -927,7 +965,7 @@ extension ClusterShell { // and we decided to tie-break and kill one of the connections. As such, the handshake COMPLETED successfully but // on the other connection; and the terminated one may yield an error (e.g. truncation error during proto parsing etc), // however that error is harmless - as we associated with the "other" right connection. - if let existingAssociation = self.getExistingAssociation(with: node), + if let existingAssociation = self.getAnyExistingAssociation(with: node), existingAssociation.isAssociated || existingAssociation.isTombstone { state.log.debug( "Handshake failed, however existing association with node exists. Could be that a concurrent handshake was failed on purpose.", @@ -950,7 +988,7 @@ extension ClusterShell { } private func onRestInPeace(_ context: ActorContext, _ state: ClusterShellState, intendedNode: UniqueNode, fromNode: UniqueNode) -> Behavior { - let myselfNode = state.myselfNode + let myselfNode = state.localNode guard myselfNode == myselfNode else { state.log.warning( @@ -1012,12 +1050,12 @@ extension ClusterShell { // FIXME: also close all associations (!!!) switch $0 { case .success: - context.log.info("Unbound server socket [\(addrDesc)], node: \(reflecting: state.myselfNode)") + context.log.info("Unbound server socket [\(addrDesc)], node: \(reflecting: state.localNode)") self.serializationPool.shutdown() signalOnceUnbound.offerOnce(()) return .stop case .failure(let err): - context.log.warning("Failed while unbinding server socket [\(addrDesc)], node: \(reflecting: state.myselfNode). Error: \(err)") + context.log.warning("Failed while unbinding server socket [\(addrDesc)], node: \(reflecting: state.localNode). Error: \(err)") self.serializationPool.shutdown() signalOnceUnbound.offerOnce(()) throw err @@ -1086,7 +1124,7 @@ extension ClusterShell { } } - guard memberToDown.node != state.myselfNode else { + guard memberToDown.node != state.localNode else { // ==== ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ // Down(self node); ensuring SWIM knows about this and should likely initiate graceful shutdown context.log.warning( diff --git a/Sources/DistributedActors/Cluster/ClusterShellState.swift b/Sources/DistributedActors/Cluster/ClusterShellState.swift index 6ab7c3071..f3d58303f 100644 --- a/Sources/DistributedActors/Cluster/ClusterShellState.swift +++ b/Sources/DistributedActors/Cluster/ClusterShellState.swift @@ -30,7 +30,7 @@ internal protocol ReadOnlyClusterState { var handshakeBackoff: BackoffStrategy { get } /// Unique address of the current node. - var myselfNode: UniqueNode { get } + var localNode: UniqueNode { get } var settings: ClusterSettings { get } } @@ -44,7 +44,7 @@ internal struct ClusterShellState: ReadOnlyClusterState { let events: EventStream - let myselfNode: UniqueNode + let localNode: UniqueNode let channel: Channel let eventLoopGroup: EventLoopGroup @@ -100,7 +100,7 @@ internal struct ClusterShellState: ReadOnlyClusterState { self.allocator = settings.allocator self.eventLoopGroup = settings.eventLoopGroup ?? settings.makeDefaultEventLoopGroup() - self.myselfNode = settings.uniqueBindNode + self.localNode = settings.uniqueBindNode self._latestGossip = Cluster.Gossip(ownerNode: settings.uniqueBindNode) self.events = events @@ -120,7 +120,7 @@ extension ClusterShellState { /// /// This MAY return `inFlight`, in which case it means someone already initiated a handshake with given node, /// and we should _do nothing_ and trust that our `whenCompleted` will be notified when the already in-flight handshake completes. - mutating func registerHandshake(with remoteNode: Node, whenCompleted: EventLoopPromise) -> HandshakeStateMachine.State { + mutating func beginHandshake(with remoteNode: Node, whenCompleted: EventLoopPromise) -> HandshakeStateMachine.State { if let handshakeState = self.handshakeInProgress(with: remoteNode) { switch handshakeState { case .initiated(let state): @@ -138,7 +138,7 @@ extension ClusterShellState { let initiated = HandshakeStateMachine.InitiatedState( settings: self.settings, - localNode: self.myselfNode, + localNode: self.localNode, connectTo: remoteNode, whenCompleted: whenCompleted ) @@ -233,16 +233,15 @@ extension ClusterShellState { /// Winner: Keeps the outgoing connection, negotiates and replies accept/reject on the "incoming" connection from the remote node. /// Loser: Drops the incoming connection and waits for Winner's decision. mutating func onIncomingHandshakeOffer(offer: Wire.HandshakeOffer, incomingChannel: Channel) -> OnIncomingHandshakeOfferDirective { - func negotiate(promise: EventLoopPromise? = nil) -> OnIncomingHandshakeOfferDirective { - let promise = promise ?? self.eventLoopGroup.next().makePromise(of: Wire.HandshakeResponse.self) + func prepareNegotiation(promise: EventLoopPromise? = nil) -> OnIncomingHandshakeOfferDirective { let fsm = HandshakeStateMachine.HandshakeReceivedState(state: self, offer: offer, whenCompleted: promise) - self._handshakes[offer.from.node] = .wasOfferedHandshake(fsm) + self._handshakes[offer.originNode.node] = .wasOfferedHandshake(fsm) return .negotiateIncoming(fsm) } - guard let inProgress = self._handshakes[offer.from.node] else { + guard let inProgress = self._handshakes[offer.originNode.node] else { // no other concurrent handshakes in progress; good, this is happy path, so we simply continue our negotiation - return negotiate() + return prepareNegotiation() } switch inProgress { @@ -254,9 +253,9 @@ extension ClusterShellState { // let selectedChannel: Channel /// order on nodes is somewhat arbitrary, but that is fine, since we only need this for tiebreakers - let tieBreakWinner = initiated.localNode < offer.from + let tieBreakWinner = initiated.localNode < offer.originNode self.log.info(""" - Concurrently initiated handshakes from nodes [\(initiated.localNode)](local) and [\(offer.from)](remote) \ + Concurrently initiated handshakes from nodes [\(initiated.localNode)](local) and [\(offer.originNode)](remote) \ detected! Resolving race by address ordering; This node \(tieBreakWinner ? "WON (will negotiate and reply)" : "LOST (will await reply)") tie-break. """, metadata: [ "handshake/inProgress": "\(initiated)", @@ -265,18 +264,17 @@ extension ClusterShellState { ]) if tieBreakWinner { - if self.closeOutboundHandshakeChannel(with: offer.from.node) != nil { + if self.closeOutboundHandshakeChannel(with: offer.originNode.node) != nil { self.log.debug( "Aborted handshake, as concurrently negotiating another one with same node already", metadata: [ "handshake/status": "abort-incoming,offer", - "handshake/from": "\(offer.from)", + "handshake/from": "\(offer.originNode)", ] ) } - self.log.debug("Proceed to negotiate handshake offer.") - return negotiate(promise: initiated.whenCompleted) + return prepareNegotiation(promise: initiated.whenCompleted) } else { // we "lost", the other node will send the accept; when it does, the will complete the future. @@ -287,7 +285,7 @@ extension ClusterShellState { // Situations: // - it could be that the remote re-sent their offer before it received our accept? // - maybe remote did not receive our accept/reject and is trying again? - return negotiate() + return prepareNegotiation() // --- these are never stored ---- case .inFlight(let inFlight): @@ -305,7 +303,7 @@ extension ClusterShellState { } mutating func incomingHandshakeAccept(_ accept: Wire.HandshakeAccept) -> HandshakeStateMachine.CompletedState? { - guard let inProgressHandshake = self._handshakes[accept.from.node] else { + guard let inProgressHandshake = self._handshakes[accept.targetNode.node] else { // TODO: what if node that sent handshake, has already terminated -- would we have removed the in progress handshake already causing this? // fatalError("Accept incoming [\(accept)] for handshake which was not in progress! On node: \(self.myselfNode), cluster shell state: \(self), membership: \(self.membership)") // TODO: model differently return nil @@ -313,7 +311,7 @@ extension ClusterShellState { switch inProgressHandshake { case .initiated(let hsm): - let completed = HandshakeStateMachine.CompletedState(fromInitiated: hsm, remoteNode: accept.from) + let completed = HandshakeStateMachine.CompletedState(fromInitiated: hsm, remoteNode: accept.targetNode) return completed case .wasOfferedHandshake: // TODO: model the states to express this can not happen // there is a client side state machine and a server side one @@ -332,7 +330,7 @@ extension ClusterShellState { /// Stores an `Association` for the newly established association; /// /// Does NOT by itself move the member to joining, but via the directive asks the outer to do this. - mutating func associate(_ system: ActorSystem, _ handshake: HandshakeStateMachine.CompletedState, channel: Channel) -> AssociatedDirective { + mutating func completeHandshakeAssociate(_ clusterShell: ClusterShell, _ handshake: HandshakeStateMachine.CompletedState, channel: Channel) -> AssociatedDirective { guard self._handshakes.removeValue(forKey: handshake.remoteNode.node) != nil else { fatalError("Can not complete a handshake which was not in progress!") // TODO: perhaps we instead just warn and ignore this; since it should be harmless @@ -353,7 +351,11 @@ extension ClusterShellState { """) } - return AssociatedDirective(membershipChange: change, handshake: handshake, channel: channel) + return AssociatedDirective( + membershipChange: change, + handshake: handshake, + channel: channel + ) } struct AssociatedDirective { @@ -419,7 +421,7 @@ extension ClusterShellState { // TODO: actions may want to be acted upon, they're like directives, we currently have no such need though; // such actions be e.g. "kill association right away" or "asap tell that node .down" directly without waiting for gossip etc - self.log.trace("Membership updated \(self.membership.prettyDescription(label: "\(self.myselfNode)")),\n by \(event)") + self.log.trace("Membership updated \(self.membership.prettyDescription(label: "\(self.localNode)")),\n by \(event)") return .init(applied: changeWasApplied, leaderActions: leaderActions) } @@ -441,13 +443,18 @@ extension ClusterShellState { ] } - func metadataForHandshakes(node: Node, error err: Error?) -> Logger.Metadata { + func metadataForHandshakes(node: Node? = nil, uniqueNode: UniqueNode? = nil, error err: Error?) -> Logger.Metadata { var metadata: Logger.Metadata = [ - "handshake/peer": "\(node)", "handshakes": "\(self.handshakes())", ] + if let n = node { + metadata["handshake/peer"] = "\(n)" + } + if let n = uniqueNode { + metadata["handshake/peer"] = "\(n)" + } if let error = err { metadata["handshake/error"] = "\(error)" metadata["handshake/errorType"] = "\(String(reflecting: type(of: error as Any)))" diff --git a/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift b/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift index fc5f2a9e2..21798c747 100644 --- a/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift +++ b/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift @@ -94,7 +94,7 @@ internal struct HandshakeStateMachine { func makeOffer() -> Wire.HandshakeOffer { // TODO: maybe store also at what time we sent the handshake, so we can diagnose if we should reject replies for being late etc - Wire.HandshakeOffer(version: self.protocolVersion, from: self.localNode, to: self.remoteNode) + Wire.HandshakeOffer(version: self.protocolVersion, originNode: self.localNode, targetNode: self.remoteNode) } mutating func onHandshakeTimeout() -> RetryDirective { @@ -154,7 +154,7 @@ internal struct HandshakeStateMachine { let offer: Wire.HandshakeOffer var boundAddress: UniqueNode { - self.state.myselfNode + self.state.localNode } var protocolVersion: DistributedActors.Version { @@ -169,52 +169,53 @@ internal struct HandshakeStateMachine { self.whenCompleted = whenCompleted } - // do not call directly, rather obtain the completed state via negotiate() - func _acceptAndMakeCompletedState() -> CompletedState { - let completed = CompletedState(fromReceived: self, remoteNode: offer.from) - self.whenCompleted?.succeed(.accept(completed.makeAccept())) - return completed - } +// // do not call directly, rather obtain the completed state via negotiate() +// func _acceptAndMakeCompletedState() -> CompletedState { +// let completed = CompletedState(fromReceived: self, remoteNode: offer.originNode) +// self.whenCompleted?.succeed(.accept(completed.makeAccept())) +// return completed +// } func negotiate() -> HandshakeStateMachine.NegotiateDirective { - guard self.boundAddress.node == self.offer.to else { + guard self.boundAddress.node == self.offer.targetNode else { let error = HandshakeError.targetHandshakeAddressMismatch(self.offer, selfNode: self.boundAddress) - let rejectedState = RejectedState(fromReceived: self, remoteNode: self.offer.from, error: error) - self.whenCompleted?.succeed(.reject(rejectedState.makeReject())) + let rejectedState = RejectedState(fromReceived: self, remoteNode: self.offer.originNode, error: error) + self.whenCompleted?.succeed(.reject(rejectedState.makeReject(whenHandshakeReplySent: { () in + self.state.log.trace("Done rejecting handshake.") // TODO: something more, terminate the association? + }))) return .rejectHandshake(rejectedState) } // negotiate version - if let negotiatedVersion = self.negotiateVersion(local: self.protocolVersion, remote: self.offer.version) { - switch negotiatedVersion { - case .rejectHandshake(let rejectedState): - self.whenCompleted?.succeed(.reject(rejectedState.makeReject())) - return negotiatedVersion - case .acceptAndAssociate: - fatalError("Should not happen, only rejections or nothing should be yielded from negotiateVersion") // TODO: more typesafety would be nice - } + if let rejectedState = self.negotiateVersion(local: self.protocolVersion, remote: self.offer.version) { + self.whenCompleted?.succeed(.reject(rejectedState.makeReject(whenHandshakeReplySent: { () in + self.state.log.trace("Done rejecting handshake.") // TODO: something more, terminate the association? + }))) + return .rejectHandshake(rejectedState) } // negotiate capabilities // self.negotiateCapabilities(...) // TODO: We may want to negotiate other options - return .acceptAndAssociate(self._acceptAndMakeCompletedState()) - } + let completed = CompletedState(fromReceived: self, remoteNode: offer.originNode) +// self.whenCompleted?.succeed(.accept(completed.makeAccept { () in +// self. +// })) +// return completed - // TODO: determine the actual logic we'd want here, for now we accept anything except major changes; use semver? - /// - Returns `rejectHandshake` or `nil` - func negotiateVersion(local: DistributedActors.Version, remote: DistributedActors.Version) -> HandshakeStateMachine.NegotiateDirective? { - let accept: HandshakeStateMachine.NegotiateDirective? = nil + return .acceptAndAssociate(completed) + } + func negotiateVersion(local: DistributedActors.Version, remote: DistributedActors.Version) -> RejectedState? { guard local.major == remote.major else { let error = HandshakeError.incompatibleProtocolVersion( local: self.protocolVersion, remote: self.offer.version ) - return .rejectHandshake(RejectedState(fromReceived: self, remoteNode: self.offer.from, error: error)) + return RejectedState(fromReceived: self, remoteNode: self.offer.originNode, error: error) } - return accept + return nil } } @@ -251,26 +252,37 @@ internal struct HandshakeStateMachine { self.whenCompleted = state.whenCompleted } - func makeAccept() -> Wire.HandshakeAccept { - .init(version: self.protocolVersion, from: self.localNode, origin: self.remoteNode) + func makeAccept(whenHandshakeReplySent: @escaping () -> Void) -> Wire.HandshakeAccept { + Wire.HandshakeAccept( + version: self.protocolVersion, + targetNode: self.localNode, + originNode: self.remoteNode, + whenHandshakeReplySent: whenHandshakeReplySent + ) } } internal struct RejectedState { let protocolVersion: Version - let localNode: Node + let localNode: UniqueNode let remoteNode: UniqueNode let error: HandshakeError init(fromReceived state: HandshakeReceivedState, remoteNode: UniqueNode, error: HandshakeError) { self.protocolVersion = state.protocolVersion - self.localNode = state.boundAddress.node + self.localNode = state.boundAddress self.remoteNode = remoteNode self.error = error } - func makeReject() -> Wire.HandshakeReject { - .init(version: self.protocolVersion, from: self.localNode, origin: self.remoteNode, reason: "\(self.error)") + func makeReject(whenHandshakeReplySent: @escaping () -> Void) -> Wire.HandshakeReject { + Wire.HandshakeReject( + version: self.protocolVersion, + targetNode: self.localNode, + originNode: self.remoteNode, + reason: "\(self.error)", + whenHandshakeReplySent: whenHandshakeReplySent + ) } } @@ -309,5 +321,7 @@ enum HandshakeError: Error { /// Returned when an incoming handshake protocol version does not match what this node can understand. case incompatibleProtocolVersion(local: DistributedActors.Version, remote: DistributedActors.Version) + case targetRejectedHandshake(selfNode: UniqueNode, remoteNode: UniqueNode, message: String) + case targetAlreadyTombstone(selfNode: UniqueNode, remoteNode: UniqueNode) } diff --git a/Sources/DistributedActors/Cluster/Transport/TransportPipelines.swift b/Sources/DistributedActors/Cluster/Transport/TransportPipelines.swift index 2a6d9f197..a4bb7cb5a 100644 --- a/Sources/DistributedActors/Cluster/Transport/TransportPipelines.swift +++ b/Sources/DistributedActors/Cluster/Transport/TransportPipelines.swift @@ -50,7 +50,7 @@ private final class InitiatingHandshakeHandler: ChannelInboundHandler, Removable } func channelRead(context: ChannelHandlerContext, data: NIOAny) { - readHandshakeResponse(context: context, bytes: self.unwrapInboundIn(data)) + self.readHandshakeResponse(context: context, bytes: self.unwrapInboundIn(data)) } private func initiateHandshake(context: ChannelHandlerContext) { @@ -80,23 +80,32 @@ private final class InitiatingHandshakeHandler: ChannelInboundHandler, Removable let response = try self.readHandshakeResponse(bytes: &bytes) switch response { case .accept(let accept): - self.log.debug("Received handshake accept from: [\(accept.from)]", metadata: metadata) + self.log.debug("Received handshake accept from: [\(accept.targetNode)]", metadata: metadata) self.cluster.tell(.inbound(.handshakeAccepted(accept, channel: context.channel))) // handshake is completed, so we remove the handler from the pipeline context.pipeline.removeHandler(self, promise: nil) case .reject(let reject): - self.log.debug("Received handshake reject from: [\(reject.from)] reason: [\(reject.reason)], closing channel.", metadata: metadata) + self.log.debug("Received handshake reject from: [\(reject.targetNode)] reason: [\(reject.reason)], closing channel.", metadata: metadata) self.cluster.tell(.inbound(.handshakeRejected(reject))) context.close(promise: nil) } } catch { self.log.debug("Handshake failure, error [\(error)]:\(String(reflecting: type(of: error)))", metadata: metadata) - self.cluster.tell(.inbound(.handshakeFailed(self.handshakeOffer.to, error))) + self.cluster.tell(.inbound(.handshakeFailed(self.handshakeOffer.targetNode, error))) _ = context.close(mode: .all) } } + + // length prefixed + private func readHandshakeResponse(bytes: inout ByteBuffer) throws -> Wire.HandshakeResponse { + guard let data = bytes.readData(length: bytes.readableBytes) else { + throw WireFormatError.notEnoughBytes(expectedAtLeastBytes: bytes.readableBytes, hint: "handshake accept") + } + let proto = try ProtoHandshakeResponse(serializedData: data) + return try Wire.HandshakeResponse(proto) + } } final class ReceivingHandshakeHandler: ChannelInboundHandler, RemovableChannelHandler { @@ -116,52 +125,76 @@ final class ReceivingHandshakeHandler: ChannelInboundHandler, RemovableChannelHa func channelRead(context: ChannelHandlerContext, data: NIOAny) { do { var bytes = self.unwrapInboundIn(data) - // TODO: formalize wire format... let offer = try self.readHandshakeOffer(bytes: &bytes) - self.log.debug("Received handshake offer from: [\(offer.from)] with protocol version: [\(offer.version)]", metadata: [ + self.log.debug("Received handshake offer from: [\(offer.originNode)] with protocol version: [\(offer.version)]", metadata: [ "handshake/channel": "\(context.channel)", ]) let promise = context.eventLoop.makePromise(of: Wire.HandshakeResponse.self) - self.cluster.tell(.inbound(.handshakeOffer(offer, channel: context.channel, replyInto: promise))) + self.cluster.tell(.inbound(.handshakeOffer(offer, channel: context.channel, handshakeReplyTo: promise))) promise.futureResult.whenComplete { res in switch res { - case .failure(let err): - context.fireErrorCaught(err) case .success(.accept(let accept)): - self.log.debug("Accepting handshake offer from: [\(offer.from)]") - let acceptProto = ProtoHandshakeAccept(accept) - var proto = ProtoHandshakeResponse() - proto.accept = acceptProto do { - let bytes = try proto.serializedByteBuffer(allocator: context.channel.allocator) - context.writeAndFlush(NIOAny(bytes), promise: nil) - // we are done with the handshake, so we can remove it and add envelope and serialization handler to process actual messages - context.pipeline.removeHandler(self, promise: nil) + self.log.debug("Accepting handshake offer from: [\(offer.originNode)]") + try self.writeHandshakeAccept(context, accept) } catch { + self.log.error("Failed when sending Handshake.Accept: \(accept), error: \(error)") context.fireErrorCaught(error) } + case .success(.reject(let reject)): - self.log.debug("Rejecting handshake offer from: [\(offer.from)] reason: [\(reject.reason)]") - let rejectProto = ProtoHandshakeReject(reject) - var proto = ProtoHandshakeResponse() - proto.reject = rejectProto do { - let bytes = try proto.serializedByteBuffer(allocator: context.channel.allocator) - context.writeAndFlush(NIOAny(bytes), promise: nil) - // we are done with the handshake, so we can remove it and add envelope and serialization handler to process actual messages - context.pipeline.removeHandler(self, promise: nil) + self.log.debug("Rejecting handshake offer from: [\(offer.originNode)] reason: [\(reject.reason)]") + try self.writeHandshakeReject(context, reject) } catch { + self.log.error("Failed when writing \(reject), error: \(error)") context.fireErrorCaught(error) } + + case .failure(let err): + context.fireErrorCaught(err) + // _ = context.close() // TODO: maybe? } } } catch { context.fireErrorCaught(error) } } + + private func readHandshakeOffer(bytes: inout ByteBuffer) throws -> Wire.HandshakeOffer { + guard let data = bytes.readData(length: bytes.readableBytes) else { + throw WireFormatError.notEnoughBytes(expectedAtLeastBytes: bytes.readableBytes, hint: "handshake offer") + } + let proto = try ProtoHandshakeOffer(serializedData: data) + return try Wire.HandshakeOffer(fromProto: proto) + } + + private func writeHandshakeAccept(_ context: ChannelHandlerContext, _ accept: Wire.HandshakeAccept) throws { + var proto = ProtoHandshakeResponse() + proto.accept = ProtoHandshakeAccept(accept) + + let bytes = try proto.serializedByteBuffer(allocator: context.channel.allocator) + context.writeAndFlush(NIOAny(bytes), promise: nil) + accept.onHandshakeReplySent?() + + // we are done with the handshake, so we can remove it and add envelope and serialization handler to process actual messages + context.pipeline.removeHandler(self, promise: nil) + } + + private func writeHandshakeReject(_ context: ChannelHandlerContext, _ reject: Wire.HandshakeReject) throws { + var proto = ProtoHandshakeResponse() + proto.reject = ProtoHandshakeReject(reject) + + let bytes = try proto.serializedByteBuffer(allocator: context.channel.allocator) + context.writeAndFlush(NIOAny(bytes), promise: nil) + reject.onHandshakeReplySent?() + + // we are done with the handshake, so we can remove it and add envelope and serialization handler to process actual messages + context.pipeline.removeHandler(self, promise: nil) + } } enum HandlerRole { @@ -606,30 +639,7 @@ private final class UserMessageHandler: ChannelInboundHandler { } } -// ==== ---------------------------------------------------------------------------------------------------------------- -// MARK: Protobuf read... implementations - -extension InitiatingHandshakeHandler { - // length prefixed - func readHandshakeResponse(bytes: inout ByteBuffer) throws -> Wire.HandshakeResponse { - guard let data = bytes.readData(length: bytes.readableBytes) else { - throw WireFormatError.notEnoughBytes(expectedAtLeastBytes: bytes.readableBytes, hint: "handshake accept") - } - let proto = try ProtoHandshakeResponse(serializedData: data) - return try Wire.HandshakeResponse(proto) - } -} - -extension ReceivingHandshakeHandler { - /// Read length prefixed data - func readHandshakeOffer(bytes: inout ByteBuffer) throws -> Wire.HandshakeOffer { - guard let data = bytes.readData(length: bytes.readableBytes) else { - throw WireFormatError.notEnoughBytes(expectedAtLeastBytes: bytes.readableBytes, hint: "handshake offer") - } - let proto = try ProtoHandshakeOffer(serializedData: data) - return try Wire.HandshakeOffer(fromProto: proto) - } -} +extension ReceivingHandshakeHandler {} private final class DumpRawBytesDebugHandler: ChannelInboundHandler { typealias InboundIn = ByteBuffer diff --git a/Sources/DistributedActors/Cluster/Transport/WireMessages.swift b/Sources/DistributedActors/Cluster/Transport/WireMessages.swift index f2e34c71d..bfa9b04ad 100644 --- a/Sources/DistributedActors/Cluster/Transport/WireMessages.swift +++ b/Sources/DistributedActors/Cluster/Transport/WireMessages.swift @@ -39,8 +39,8 @@ internal enum Wire { internal struct HandshakeOffer: WireMessage { internal var version: Version - internal var from: UniqueNode - internal var to: Node + internal var originNode: UniqueNode + internal var targetNode: Node } internal enum HandshakeResponse: WireMessage { @@ -63,19 +63,23 @@ internal enum Wire { /// The node accepting the handshake. /// /// This will always be the "local" node where the accept is being made. - internal let from: UniqueNode + internal let targetNode: UniqueNode /// In order to avoid confusion with from/to, we name the `origin` the node which an *offer* was sent from, /// and we now reply to this handshake to it. This value is carried so the origin can confirm it indeed was /// intended for it, and not a previous incarnation of a system on the same network address. /// /// This will always be the "remote" node, with regards to where the accept is created. - internal let origin: UniqueNode + internal let originNode: UniqueNode - init(version: Version, from: UniqueNode, origin: UniqueNode) { + /// MUST be called after the reply is written to the wire; triggers messages being flushed from the association. + internal var onHandshakeReplySent: (() -> Void)? + + init(version: Version, targetNode: UniqueNode, originNode: UniqueNode, whenHandshakeReplySent: (() -> Void)?) { self.version = version - self.from = from - self.origin = origin + self.targetNode = targetNode + self.originNode = originNode + self.onHandshakeReplySent = whenHandshakeReplySent } } @@ -84,15 +88,18 @@ internal enum Wire { internal let version: Version internal let reason: String - /// not a UniqueNode, so we can't proceed into establishing an association - even by accident - internal let from: Node - internal let origin: UniqueNode + internal let targetNode: UniqueNode + internal let originNode: UniqueNode + + /// MUST be called after the reply is written to the wire; triggers messages being flushed from the association. + internal let onHandshakeReplySent: (() -> Void)? - init(version: Wire.Version, from: Node, origin: UniqueNode, reason: String) { + init(version: Wire.Version, targetNode: UniqueNode, originNode: UniqueNode, reason: String, whenHandshakeReplySent: (() -> Void)?) { self.version = version - self.from = from - self.origin = origin + self.targetNode = targetNode + self.originNode = originNode self.reason = reason + self.onHandshakeReplySent = whenHandshakeReplySent } } } diff --git a/Sources/DistributedActors/Protobuf/WireProtocol+Serialization.swift b/Sources/DistributedActors/Protobuf/WireProtocol+Serialization.swift index af2c7fe16..afa9658b2 100644 --- a/Sources/DistributedActors/Protobuf/WireProtocol+Serialization.swift +++ b/Sources/DistributedActors/Protobuf/WireProtocol+Serialization.swift @@ -24,7 +24,7 @@ enum WireEnvelopeError: Error { case emptyRecipient } -// TODO: ProtobufRepresentable? +// TODO: Implement these using ProtobufRepresentable extension Wire.Envelope { init(_ proto: ProtoEnvelope) throws { @@ -86,23 +86,24 @@ extension Wire.HandshakeAccept { guard proto.hasVersion else { throw SerializationError.missingField("version", type: String(describing: Wire.HandshakeAccept.self)) } - guard proto.hasFrom else { - throw SerializationError.missingField("from", type: String(describing: Wire.HandshakeAccept.self)) + guard proto.hasTargetNode else { + throw SerializationError.missingField("targetNode", type: String(describing: Wire.HandshakeAccept.self)) } - guard proto.hasOrigin else { - throw SerializationError.missingField("hasOrigin", type: String(describing: Wire.HandshakeAccept.self)) + guard proto.hasOriginNode else { + throw SerializationError.missingField("originNode", type: String(describing: Wire.HandshakeAccept.self)) } self.version = .init(proto.version) - self.from = try .init(proto.from) - self.origin = try .init(proto.origin) + self.targetNode = try .init(proto.targetNode) + self.originNode = try .init(proto.originNode) + self.onHandshakeReplySent = nil } } extension ProtoHandshakeAccept { init(_ accept: Wire.HandshakeAccept) { self.version = .init(accept.version) - self.from = .init(accept.from) - self.origin = .init(accept.origin) + self.targetNode = .init(accept.targetNode) + self.originNode = .init(accept.originNode) } } @@ -114,16 +115,17 @@ extension Wire.HandshakeReject { guard proto.hasVersion else { throw SerializationError.missingField("version", type: String(describing: Wire.HandshakeReject.self)) } - guard proto.hasFrom else { - throw SerializationError.missingField("from", type: String(describing: Wire.HandshakeReject.self)) + guard proto.hasTargetNode else { + throw SerializationError.missingField("targetNode", type: String(describing: Wire.HandshakeReject.self)) } - guard proto.hasOrigin else { - throw SerializationError.missingField("origin", type: String(describing: Wire.HandshakeReject.self)) + guard proto.hasOriginNode else { + throw SerializationError.missingField("originNode", type: String(describing: Wire.HandshakeReject.self)) } self.version = .init(proto.version) - self.from = .init(proto.from) - self.origin = try .init(proto.origin) + self.targetNode = try .init(proto.targetNode) + self.originNode = try .init(proto.originNode) + self.onHandshakeReplySent = nil self.reason = proto.reason } } @@ -131,8 +133,8 @@ extension Wire.HandshakeReject { extension ProtoHandshakeReject { init(_ reject: Wire.HandshakeReject) { self.version = .init(reject.version) - self.from = .init(reject.from) - self.origin = .init(reject.origin) + self.targetNode = .init(reject.targetNode) + self.originNode = .init(reject.originNode) self.reason = reject.reason } } @@ -142,18 +144,18 @@ extension ProtoHandshakeReject { extension Wire.HandshakeOffer { init(fromProto proto: ProtoHandshakeOffer) throws { - guard proto.hasFrom else { - throw SerializationError.missingField("from", type: String(reflecting: Wire.HandshakeOffer.self)) + guard proto.hasOriginNode else { + throw SerializationError.missingField("originNode", type: String(reflecting: Wire.HandshakeOffer.self)) } - guard proto.hasTo else { - throw SerializationError.missingField("to", type: String(reflecting: Wire.HandshakeOffer.self)) + guard proto.hasTargetNode else { + throw SerializationError.missingField("targetNode", type: String(reflecting: Wire.HandshakeOffer.self)) } guard proto.hasVersion else { throw SerializationError.missingField("version", type: String(reflecting: Wire.HandshakeOffer.self)) } - self.from = try UniqueNode(proto.from) - self.to = Node(proto.to) + self.originNode = try UniqueNode(proto.originNode) + self.targetNode = Node(proto.targetNode) self.version = Wire.Version(reserved: UInt8(proto.version.reserved), major: UInt8(proto.version.major), minor: UInt8(proto.version.minor), patch: UInt8(proto.version.patch)) } } @@ -161,8 +163,8 @@ extension Wire.HandshakeOffer { extension ProtoHandshakeOffer { init(_ offer: Wire.HandshakeOffer) { self.version = ProtoProtocolVersion(offer.version) - self.from = ProtoUniqueNode(offer.from) - self.to = ProtoNode(offer.to) + self.originNode = ProtoUniqueNode(offer.originNode) + self.targetNode = ProtoNode(offer.targetNode) } init(serializedData data: Data) throws { @@ -172,11 +174,11 @@ extension ProtoHandshakeOffer { guard proto.hasVersion else { throw SerializationError.missingField("version", type: String(reflecting: Wire.HandshakeOffer.self)) } - guard proto.hasFrom else { - throw SerializationError.missingField("from", type: String(reflecting: Wire.HandshakeOffer.self)) + guard proto.hasOriginNode else { + throw SerializationError.missingField("hasOriginNode", type: String(reflecting: Wire.HandshakeOffer.self)) } - guard proto.hasTo else { - throw SerializationError.missingField("to", type: String(reflecting: Wire.HandshakeOffer.self)) + guard proto.hasTargetNode else { + throw SerializationError.missingField("targetNode", type: String(reflecting: Wire.HandshakeOffer.self)) } self = proto diff --git a/Sources/DistributedActors/Protobuf/WireProtocol.pb.swift b/Sources/DistributedActors/Protobuf/WireProtocol.pb.swift index 533f16bb9..b51037739 100644 --- a/Sources/DistributedActors/Protobuf/WireProtocol.pb.swift +++ b/Sources/DistributedActors/Protobuf/WireProtocol.pb.swift @@ -47,28 +47,28 @@ struct ProtoHandshakeOffer { /// Clears the value of `version`. Subsequent reads from it will return its default value. mutating func clearVersion() {_uniqueStorage()._version = nil} - var from: ProtoUniqueNode { - get {return _storage._from ?? ProtoUniqueNode()} - set {_uniqueStorage()._from = newValue} + var originNode: ProtoUniqueNode { + get {return _storage._originNode ?? ProtoUniqueNode()} + set {_uniqueStorage()._originNode = newValue} } - /// Returns true if `from` has been explicitly set. - var hasFrom: Bool {return _storage._from != nil} - /// Clears the value of `from`. Subsequent reads from it will return its default value. - mutating func clearFrom() {_uniqueStorage()._from = nil} + /// Returns true if `originNode` has been explicitly set. + var hasOriginNode: Bool {return _storage._originNode != nil} + /// Clears the value of `originNode`. Subsequent reads from it will return its default value. + mutating func clearOriginNode() {_uniqueStorage()._originNode = nil} /// In the future we may want to add additional information /// about certain capabilities here. E.g. when a node supports /// faster transport like InfiniBand and the likes, so we can /// upgrade the connection in case both nodes support the fast /// transport. - var to: ProtoNode { - get {return _storage._to ?? ProtoNode()} - set {_uniqueStorage()._to = newValue} + var targetNode: ProtoNode { + get {return _storage._targetNode ?? ProtoNode()} + set {_uniqueStorage()._targetNode = newValue} } - /// Returns true if `to` has been explicitly set. - var hasTo: Bool {return _storage._to != nil} - /// Clears the value of `to`. Subsequent reads from it will return its default value. - mutating func clearTo() {_uniqueStorage()._to = nil} + /// Returns true if `targetNode` has been explicitly set. + var hasTargetNode: Bool {return _storage._targetNode != nil} + /// Clears the value of `targetNode`. Subsequent reads from it will return its default value. + mutating func clearTargetNode() {_uniqueStorage()._targetNode = nil} var unknownFields = SwiftProtobuf.UnknownStorage() @@ -139,23 +139,23 @@ struct ProtoHandshakeAccept { /// Clears the value of `version`. Subsequent reads from it will return its default value. mutating func clearVersion() {_uniqueStorage()._version = nil} - var origin: ProtoUniqueNode { - get {return _storage._origin ?? ProtoUniqueNode()} - set {_uniqueStorage()._origin = newValue} + var originNode: ProtoUniqueNode { + get {return _storage._originNode ?? ProtoUniqueNode()} + set {_uniqueStorage()._originNode = newValue} } - /// Returns true if `origin` has been explicitly set. - var hasOrigin: Bool {return _storage._origin != nil} - /// Clears the value of `origin`. Subsequent reads from it will return its default value. - mutating func clearOrigin() {_uniqueStorage()._origin = nil} + /// Returns true if `originNode` has been explicitly set. + var hasOriginNode: Bool {return _storage._originNode != nil} + /// Clears the value of `originNode`. Subsequent reads from it will return its default value. + mutating func clearOriginNode() {_uniqueStorage()._originNode = nil} - var from: ProtoUniqueNode { - get {return _storage._from ?? ProtoUniqueNode()} - set {_uniqueStorage()._from = newValue} + var targetNode: ProtoUniqueNode { + get {return _storage._targetNode ?? ProtoUniqueNode()} + set {_uniqueStorage()._targetNode = newValue} } - /// Returns true if `from` has been explicitly set. - var hasFrom: Bool {return _storage._from != nil} - /// Clears the value of `from`. Subsequent reads from it will return its default value. - mutating func clearFrom() {_uniqueStorage()._from = nil} + /// Returns true if `targetNode` has been explicitly set. + var hasTargetNode: Bool {return _storage._targetNode != nil} + /// Clears the value of `targetNode`. Subsequent reads from it will return its default value. + mutating func clearTargetNode() {_uniqueStorage()._targetNode = nil} var unknownFields = SwiftProtobuf.UnknownStorage() @@ -178,25 +178,23 @@ struct ProtoHandshakeReject { /// Clears the value of `version`. Subsequent reads from it will return its default value. mutating func clearVersion() {_uniqueStorage()._version = nil} - var origin: ProtoUniqueNode { - get {return _storage._origin ?? ProtoUniqueNode()} - set {_uniqueStorage()._origin = newValue} + var originNode: ProtoUniqueNode { + get {return _storage._originNode ?? ProtoUniqueNode()} + set {_uniqueStorage()._originNode = newValue} } - /// Returns true if `origin` has been explicitly set. - var hasOrigin: Bool {return _storage._origin != nil} - /// Clears the value of `origin`. Subsequent reads from it will return its default value. - mutating func clearOrigin() {_uniqueStorage()._origin = nil} + /// Returns true if `originNode` has been explicitly set. + var hasOriginNode: Bool {return _storage._originNode != nil} + /// Clears the value of `originNode`. Subsequent reads from it will return its default value. + mutating func clearOriginNode() {_uniqueStorage()._originNode = nil} - /// In the reject case this is an `Node` instead of a `UniqueNode`, - /// to explicitly prevent this from forming an association. - var from: ProtoNode { - get {return _storage._from ?? ProtoNode()} - set {_uniqueStorage()._from = newValue} + var targetNode: ProtoUniqueNode { + get {return _storage._targetNode ?? ProtoUniqueNode()} + set {_uniqueStorage()._targetNode = newValue} } - /// Returns true if `from` has been explicitly set. - var hasFrom: Bool {return _storage._from != nil} - /// Clears the value of `from`. Subsequent reads from it will return its default value. - mutating func clearFrom() {_uniqueStorage()._from = nil} + /// Returns true if `targetNode` has been explicitly set. + var hasTargetNode: Bool {return _storage._targetNode != nil} + /// Clears the value of `targetNode`. Subsequent reads from it will return its default value. + mutating func clearTargetNode() {_uniqueStorage()._targetNode = nil} var reason: String { get {return _storage._reason} @@ -346,14 +344,14 @@ extension ProtoHandshakeOffer: SwiftProtobuf.Message, SwiftProtobuf._MessageImpl static let protoMessageName: String = "HandshakeOffer" static let _protobuf_nameMap: SwiftProtobuf._NameMap = [ 1: .same(proto: "version"), - 2: .same(proto: "from"), - 3: .same(proto: "to"), + 2: .same(proto: "originNode"), + 3: .same(proto: "targetNode"), ] fileprivate class _StorageClass { var _version: ProtoProtocolVersion? = nil - var _from: ProtoUniqueNode? = nil - var _to: ProtoNode? = nil + var _originNode: ProtoUniqueNode? = nil + var _targetNode: ProtoNode? = nil static let defaultInstance = _StorageClass() @@ -361,8 +359,8 @@ extension ProtoHandshakeOffer: SwiftProtobuf.Message, SwiftProtobuf._MessageImpl init(copying source: _StorageClass) { _version = source._version - _from = source._from - _to = source._to + _originNode = source._originNode + _targetNode = source._targetNode } } @@ -379,8 +377,8 @@ extension ProtoHandshakeOffer: SwiftProtobuf.Message, SwiftProtobuf._MessageImpl while let fieldNumber = try decoder.nextFieldNumber() { switch fieldNumber { case 1: try decoder.decodeSingularMessageField(value: &_storage._version) - case 2: try decoder.decodeSingularMessageField(value: &_storage._from) - case 3: try decoder.decodeSingularMessageField(value: &_storage._to) + case 2: try decoder.decodeSingularMessageField(value: &_storage._originNode) + case 3: try decoder.decodeSingularMessageField(value: &_storage._targetNode) default: break } } @@ -392,10 +390,10 @@ extension ProtoHandshakeOffer: SwiftProtobuf.Message, SwiftProtobuf._MessageImpl if let v = _storage._version { try visitor.visitSingularMessageField(value: v, fieldNumber: 1) } - if let v = _storage._from { + if let v = _storage._originNode { try visitor.visitSingularMessageField(value: v, fieldNumber: 2) } - if let v = _storage._to { + if let v = _storage._targetNode { try visitor.visitSingularMessageField(value: v, fieldNumber: 3) } } @@ -408,8 +406,8 @@ extension ProtoHandshakeOffer: SwiftProtobuf.Message, SwiftProtobuf._MessageImpl let _storage = _args.0 let rhs_storage = _args.1 if _storage._version != rhs_storage._version {return false} - if _storage._from != rhs_storage._from {return false} - if _storage._to != rhs_storage._to {return false} + if _storage._originNode != rhs_storage._originNode {return false} + if _storage._targetNode != rhs_storage._targetNode {return false} return true } if !storagesAreEqual {return false} @@ -504,14 +502,14 @@ extension ProtoHandshakeAccept: SwiftProtobuf.Message, SwiftProtobuf._MessageImp static let protoMessageName: String = "HandshakeAccept" static let _protobuf_nameMap: SwiftProtobuf._NameMap = [ 1: .same(proto: "version"), - 2: .same(proto: "origin"), - 3: .same(proto: "from"), + 2: .same(proto: "originNode"), + 3: .same(proto: "targetNode"), ] fileprivate class _StorageClass { var _version: ProtoProtocolVersion? = nil - var _origin: ProtoUniqueNode? = nil - var _from: ProtoUniqueNode? = nil + var _originNode: ProtoUniqueNode? = nil + var _targetNode: ProtoUniqueNode? = nil static let defaultInstance = _StorageClass() @@ -519,8 +517,8 @@ extension ProtoHandshakeAccept: SwiftProtobuf.Message, SwiftProtobuf._MessageImp init(copying source: _StorageClass) { _version = source._version - _origin = source._origin - _from = source._from + _originNode = source._originNode + _targetNode = source._targetNode } } @@ -537,8 +535,8 @@ extension ProtoHandshakeAccept: SwiftProtobuf.Message, SwiftProtobuf._MessageImp while let fieldNumber = try decoder.nextFieldNumber() { switch fieldNumber { case 1: try decoder.decodeSingularMessageField(value: &_storage._version) - case 2: try decoder.decodeSingularMessageField(value: &_storage._origin) - case 3: try decoder.decodeSingularMessageField(value: &_storage._from) + case 2: try decoder.decodeSingularMessageField(value: &_storage._originNode) + case 3: try decoder.decodeSingularMessageField(value: &_storage._targetNode) default: break } } @@ -550,10 +548,10 @@ extension ProtoHandshakeAccept: SwiftProtobuf.Message, SwiftProtobuf._MessageImp if let v = _storage._version { try visitor.visitSingularMessageField(value: v, fieldNumber: 1) } - if let v = _storage._origin { + if let v = _storage._originNode { try visitor.visitSingularMessageField(value: v, fieldNumber: 2) } - if let v = _storage._from { + if let v = _storage._targetNode { try visitor.visitSingularMessageField(value: v, fieldNumber: 3) } } @@ -566,8 +564,8 @@ extension ProtoHandshakeAccept: SwiftProtobuf.Message, SwiftProtobuf._MessageImp let _storage = _args.0 let rhs_storage = _args.1 if _storage._version != rhs_storage._version {return false} - if _storage._origin != rhs_storage._origin {return false} - if _storage._from != rhs_storage._from {return false} + if _storage._originNode != rhs_storage._originNode {return false} + if _storage._targetNode != rhs_storage._targetNode {return false} return true } if !storagesAreEqual {return false} @@ -581,15 +579,15 @@ extension ProtoHandshakeReject: SwiftProtobuf.Message, SwiftProtobuf._MessageImp static let protoMessageName: String = "HandshakeReject" static let _protobuf_nameMap: SwiftProtobuf._NameMap = [ 1: .same(proto: "version"), - 2: .same(proto: "origin"), - 3: .same(proto: "from"), + 2: .same(proto: "originNode"), + 3: .same(proto: "targetNode"), 4: .same(proto: "reason"), ] fileprivate class _StorageClass { var _version: ProtoProtocolVersion? = nil - var _origin: ProtoUniqueNode? = nil - var _from: ProtoNode? = nil + var _originNode: ProtoUniqueNode? = nil + var _targetNode: ProtoUniqueNode? = nil var _reason: String = String() static let defaultInstance = _StorageClass() @@ -598,8 +596,8 @@ extension ProtoHandshakeReject: SwiftProtobuf.Message, SwiftProtobuf._MessageImp init(copying source: _StorageClass) { _version = source._version - _origin = source._origin - _from = source._from + _originNode = source._originNode + _targetNode = source._targetNode _reason = source._reason } } @@ -617,8 +615,8 @@ extension ProtoHandshakeReject: SwiftProtobuf.Message, SwiftProtobuf._MessageImp while let fieldNumber = try decoder.nextFieldNumber() { switch fieldNumber { case 1: try decoder.decodeSingularMessageField(value: &_storage._version) - case 2: try decoder.decodeSingularMessageField(value: &_storage._origin) - case 3: try decoder.decodeSingularMessageField(value: &_storage._from) + case 2: try decoder.decodeSingularMessageField(value: &_storage._originNode) + case 3: try decoder.decodeSingularMessageField(value: &_storage._targetNode) case 4: try decoder.decodeSingularStringField(value: &_storage._reason) default: break } @@ -631,10 +629,10 @@ extension ProtoHandshakeReject: SwiftProtobuf.Message, SwiftProtobuf._MessageImp if let v = _storage._version { try visitor.visitSingularMessageField(value: v, fieldNumber: 1) } - if let v = _storage._origin { + if let v = _storage._originNode { try visitor.visitSingularMessageField(value: v, fieldNumber: 2) } - if let v = _storage._from { + if let v = _storage._targetNode { try visitor.visitSingularMessageField(value: v, fieldNumber: 3) } if !_storage._reason.isEmpty { @@ -650,8 +648,8 @@ extension ProtoHandshakeReject: SwiftProtobuf.Message, SwiftProtobuf._MessageImp let _storage = _args.0 let rhs_storage = _args.1 if _storage._version != rhs_storage._version {return false} - if _storage._origin != rhs_storage._origin {return false} - if _storage._from != rhs_storage._from {return false} + if _storage._originNode != rhs_storage._originNode {return false} + if _storage._targetNode != rhs_storage._targetNode {return false} if _storage._reason != rhs_storage._reason {return false} return true } diff --git a/Sources/DistributedActorsTestKit/Cluster/ClusteredNodesTestBase.swift b/Sources/DistributedActorsTestKit/Cluster/ClusteredNodesTestBase.swift index afba83323..ee053ad67 100644 --- a/Sources/DistributedActorsTestKit/Cluster/ClusteredNodesTestBase.swift +++ b/Sources/DistributedActorsTestKit/Cluster/ClusteredNodesTestBase.swift @@ -137,7 +137,7 @@ open class ClusteredNodesTestBase: XCTestCase { } public func ensureNodes( - _ status: Cluster.MemberStatus, on system: ActorSystem? = nil, within: TimeAmount = .seconds(10), nodes: UniqueNode..., + _ status: Cluster.MemberStatus, on system: ActorSystem? = nil, within: TimeAmount = .seconds(15), nodes: UniqueNode..., file: StaticString = #file, line: UInt = #line ) throws { guard let onSystem = system ?? self._nodes.first(where: { !$0.isShuttingDown }) else { diff --git a/Tests/ActorSingletonPluginTests/ActorSingletonPluginClusteredTests.swift b/Tests/ActorSingletonPluginTests/ActorSingletonPluginClusteredTests.swift index 5b4aeed43..464028bb0 100644 --- a/Tests/ActorSingletonPluginTests/ActorSingletonPluginClusteredTests.swift +++ b/Tests/ActorSingletonPluginTests/ActorSingletonPluginClusteredTests.swift @@ -18,7 +18,19 @@ import DistributedActorsTestKit import XCTest final class ActorSingletonPluginClusteredTests: ClusteredNodesTestBase { - func test_singletonByClusterLeadership() throws { + override func configureLogCapture(settings: inout LogCapture.Settings) { + settings.excludeActorPaths = [ + "/system/cluster/swim", + "/system/cluster", + "/system/cluster/gossip", + ] + } + +// override var alwaysPrintCaptureLogs: Bool { +// true +// } + + func test_singletonByClusterLeadership_happyPath() throws { try shouldNotThrow { var singletonSettings = ActorSingletonSettings(name: GreeterSingleton.name) singletonSettings.allocationStrategy = .byLeadership @@ -51,23 +63,14 @@ final class ActorSingletonPluginClusteredTests: ClusteredNodesTestBase { let ref3 = try third.singleton.host(GreeterSingleton.Message.self, settings: singletonSettings, GreeterSingleton.makeBehavior(instance: GreeterSingleton("Hello-3"))) first.cluster.join(node: second.cluster.node.node) - third.cluster.join(node: second.cluster.node.node) + third.cluster.join(node: first.cluster.node.node) // `first` will be the leader (lowest address) and runs the singleton - try self.ensureNodes(.up, within: .seconds(15), nodes: first.cluster.node, second.cluster.node, third.cluster.node) - - let replyProbe1 = self.testKit(first).spawnTestProbe(expecting: String.self) - ref1.tell(.greet(name: "Charlie", _replyTo: replyProbe1.ref)) - - let replyProbe2 = self.testKit(second).spawnTestProbe(expecting: String.self) - ref2.tell(.greet(name: "Charlie", _replyTo: replyProbe2.ref)) - - let replyProbe3 = self.testKit(third).spawnTestProbe(expecting: String.self) - ref3.tell(.greet(name: "Charlie", _replyTo: replyProbe3.ref)) + try self.ensureNodes(.up, nodes: first.cluster.node, second.cluster.node, third.cluster.node) - try replyProbe1.expectMessage("Hello-1 Charlie!") - try replyProbe2.expectMessage("Hello-1 Charlie!") - try replyProbe3.expectMessage("Hello-1 Charlie!") + try self.assertSingletonRequestReply(first, singletonRef: ref1, message: "Alice", expect: "Hello-1 Alice!") + try self.assertSingletonRequestReply(second, singletonRef: ref2, message: "Bob", expect: "Hello-1 Bob!") + try self.assertSingletonRequestReply(third, singletonRef: ref3, message: "Charlie", expect: "Hello-1 Charlie!") } } @@ -101,11 +104,11 @@ final class ActorSingletonPluginClusteredTests: ClusteredNodesTestBase { // No leader so singleton is not available, messages sent should be stashed let replyProbe1 = self.testKit(first).spawnTestProbe(expecting: String.self) let ref1 = try first.singleton.host(GreeterSingleton.Message.self, settings: singletonSettings, GreeterSingleton.makeBehavior(instance: GreeterSingleton("Hello-1"))) - ref1.tell(.greet(name: "Charlie-1", _replyTo: replyProbe1.ref)) + ref1.tell(.greet(name: "Alice-1", _replyTo: replyProbe1.ref)) let replyProbe2 = self.testKit(second).spawnTestProbe(expecting: String.self) let ref2 = try second.singleton.host(GreeterSingleton.Message.self, settings: singletonSettings, GreeterSingleton.makeBehavior(instance: GreeterSingleton("Hello-2"))) - ref2.tell(.greet(name: "Charlie-2", _replyTo: replyProbe2.ref)) + ref2.tell(.greet(name: "Bob-2", _replyTo: replyProbe2.ref)) let replyProbe3 = self.testKit(third).spawnTestProbe(expecting: String.self) let ref3 = try third.singleton.host(GreeterSingleton.Message.self, settings: singletonSettings, GreeterSingleton.makeBehavior(instance: GreeterSingleton("Hello-3"))) @@ -119,10 +122,10 @@ final class ActorSingletonPluginClusteredTests: ClusteredNodesTestBase { third.cluster.join(node: second.cluster.node.node) // `first` becomes the leader (lowest address) and runs the singleton - try self.ensureNodes(.up, within: .seconds(15), nodes: first.cluster.node, second.cluster.node, third.cluster.node) + try self.ensureNodes(.up, nodes: first.cluster.node, second.cluster.node, third.cluster.node) - try replyProbe1.expectMessage("Hello-1 Charlie-1!") - try replyProbe2.expectMessage("Hello-1 Charlie-2!") + try replyProbe1.expectMessage("Hello-1 Alice-1!") + try replyProbe2.expectMessage("Hello-1 Bob-2!") try replyProbe3.expectMessage("Hello-1 Charlie-3!") } } @@ -170,25 +173,21 @@ final class ActorSingletonPluginClusteredTests: ClusteredNodesTestBase { first.cluster.join(node: second.cluster.node.node) third.cluster.join(node: second.cluster.node.node) - try self.ensureNodes(.up, within: .seconds(15), nodes: first.cluster.node, second.cluster.node, third.cluster.node) - - let replyProbe1 = self.testKit(first).spawnTestProbe(expecting: String.self) - ref1.tell(.greet(name: "Alice", _replyTo: replyProbe1.ref)) + try self.ensureNodes(.up, nodes: first.cluster.node, second.cluster.node, third.cluster.node) + pinfo("Nodes up: \([first.cluster.node, second.cluster.node, third.cluster.node])") let replyProbe2 = self.testKit(second).spawnTestProbe(expecting: String.self) - ref2.tell(.greet(name: "Bob", _replyTo: replyProbe2.ref)) - let replyProbe3 = self.testKit(third).spawnTestProbe(expecting: String.self) - ref3.tell(.greet(name: "Charlie", _replyTo: replyProbe3.ref)) // `first` has the lowest address so it should be the leader and singleton - try replyProbe1.expectMessage("Hello-1 Alice!") - try replyProbe2.expectMessage("Hello-1 Bob!") - try replyProbe3.expectMessage("Hello-1 Charlie!") + try self.assertSingletonRequestReply(first, singletonRef: ref1, message: "Alice", expect: "Hello-1 Alice!") + try self.assertSingletonRequestReply(second, singletonRef: ref2, message: "Bob", expect: "Hello-1 Bob!") + try self.assertSingletonRequestReply(third, singletonRef: ref3, message: "Charlie", expect: "Hello-1 Charlie!") + pinfo("All three nodes communicated with singleton") - // Take down the leader let firstNode = first.cluster.node first.cluster.leave() + pinfo("Node \(first.cluster.node) left cluster...") // Make sure that `second` and `third` see `first` as down and become leader-less try self.testKit(second).eventually(within: .seconds(10)) { @@ -209,11 +208,36 @@ final class ActorSingletonPluginClusteredTests: ClusteredNodesTestBase { // `fourth` will become the new leader and singleton fourth.cluster.join(node: second.cluster.node.node) - try self.ensureNodes(.up, on: second, within: .seconds(10), nodes: fourth.cluster.node, second.cluster.node, third.cluster.node) + try self.ensureNodes(.up, on: second, nodes: fourth.cluster.node, second.cluster.node, third.cluster.node) + pinfo("Fourth node joined, will become leader; Members now: \([fourth.cluster.node, second.cluster.node, third.cluster.node])") // The stashed messages get routed to new singleton running on `fourth` try replyProbe2.expectMessage("Hello-4 Bob-2!") try replyProbe3.expectMessage("Hello-4 Charlie-3!") + + pinfo("Nodes communicated successfully with singleton on [fourth]") + } + } + + /// Since during re-balancing it may happen that a message gets lost, we send messages a few times and only if none "got through" it would be a serious error. + private func assertSingletonRequestReply(_ system: ActorSystem, singletonRef: ActorRef, message: String, expect: String) throws { + let testKit: ActorTestKit = self.testKit(system) + let replyProbe = testKit.spawnTestProbe(expecting: String.self) + + var attempts = 0 + try testKit.eventually(within: .seconds(10)) { + attempts += 1 + singletonRef.tell(.greet(name: message, _replyTo: replyProbe.ref)) + + if let message = try replyProbe.maybeExpectMessage() { + message.shouldEqual(expect) + } else { + throw TestError( + """ + Received no reply from singleton [\(singletonRef)] while sending from [\(system.cluster.node.node)], \ + perhaps request was lost. Sent [\(message)] and expected: [\(expect)] (attempts: \(attempts)) + """) + } } } } diff --git a/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift b/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift index 80c704cf2..556aa81b5 100644 --- a/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift +++ b/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift @@ -158,38 +158,38 @@ final class ClusterAssociationTests: ClusteredNodesTestBase { // MARK: Concurrently initiated handshakes to same node should both get completed func test_association_shouldEstablishSingleAssociationForConcurrentlyInitiatedHandshakes_incoming_outgoing() throws { - let (local, remote) = self.setUpPair() + let (first, second) = self.setUpPair() - let p7337 = self.testKit(local).spawnTestProbe(expecting: ClusterShell.HandshakeResult.self) - let p8228 = self.testKit(remote).spawnTestProbe(expecting: ClusterShell.HandshakeResult.self) + let firstProbe = self.testKit(first).spawnTestProbe(expecting: ClusterShell.HandshakeResult.self) + let secondProbe = self.testKit(second).spawnTestProbe(expecting: ClusterShell.HandshakeResult.self) // here we attempt to make a race where the nodes race to join each other // again, only one association should be created. - local.cluster.ref.tell(.command(.handshakeWith(remote.cluster.node.node, replyTo: p7337.ref))) - remote.cluster.ref.tell(.command(.handshakeWith(local.cluster.node.node, replyTo: p8228.ref))) + first.cluster.ref.tell(.command(.handshakeWith(second.cluster.node.node, replyTo: firstProbe.ref))) + second.cluster.ref.tell(.command(.handshakeWith(first.cluster.node.node, replyTo: secondProbe.ref))) - _ = try p7337.expectMessage() - _ = try p8228.expectMessage() +// _ = try firstProbe.expectMessage() +// _ = try secondProbe.expectMessage() - try assertAssociated(local, withExactly: remote.settings.cluster.uniqueBindNode) - try assertAssociated(remote, withExactly: local.settings.cluster.uniqueBindNode) + try assertAssociated(first, withExactly: second.settings.cluster.uniqueBindNode) + try assertAssociated(second, withExactly: first.settings.cluster.uniqueBindNode) } func test_association_shouldEstablishSingleAssociationForConcurrentlyInitiatedHandshakes_outgoing_outgoing() throws { - let (local, remote) = setUpPair() + let (first, second) = setUpPair() - let p7337 = self.testKit(local).spawnTestProbe(expecting: ClusterShell.HandshakeResult.self) - let p8228 = self.testKit(local).spawnTestProbe(expecting: ClusterShell.HandshakeResult.self) + let firstProbe = self.testKit(first).spawnTestProbe(expecting: ClusterShell.HandshakeResult.self) + let secondProbe = self.testKit(first).spawnTestProbe(expecting: ClusterShell.HandshakeResult.self) // we issue two handshakes quickly after each other, both should succeed but there should only be one association established (!) - local.cluster.ref.tell(.command(.handshakeWith(remote.cluster.node.node, replyTo: p7337.ref))) - local.cluster.ref.tell(.command(.handshakeWith(remote.cluster.node.node, replyTo: p8228.ref))) + first.cluster.ref.tell(.command(.handshakeWith(second.cluster.node.node, replyTo: firstProbe.ref))) + first.cluster.ref.tell(.command(.handshakeWith(second.cluster.node.node, replyTo: secondProbe.ref))) - _ = try p7337.expectMessage() - _ = try p8228.expectMessage() + _ = try firstProbe.expectMessage() + _ = try secondProbe.expectMessage() - try assertAssociated(local, withExactly: remote.settings.cluster.uniqueBindNode) - try assertAssociated(remote, withExactly: local.settings.cluster.uniqueBindNode) + try assertAssociated(first, withExactly: second.settings.cluster.uniqueBindNode) + try assertAssociated(second, withExactly: first.settings.cluster.uniqueBindNode) } // ==== ------------------------------------------------------------------------------------------------------------ diff --git a/Tests/DistributedActorsTests/Cluster/ClusterLeaderActionsTests.swift b/Tests/DistributedActorsTests/Cluster/ClusterLeaderActionsTests.swift index 0c8850b64..1e5e5bf31 100644 --- a/Tests/DistributedActorsTests/Cluster/ClusterLeaderActionsTests.swift +++ b/Tests/DistributedActorsTests/Cluster/ClusterLeaderActionsTests.swift @@ -33,15 +33,15 @@ final class ClusterLeaderActionsTests: XCTestCase { var stateC: ClusterShellState! var nodeA: UniqueNode { - self.stateA.myselfNode + self.stateA.localNode } var nodeB: UniqueNode { - self.stateB.myselfNode + self.stateB.localNode } var nodeC: UniqueNode { - self.stateC.myselfNode + self.stateC.localNode } override func setUp() { diff --git a/Tests/DistributedActorsTests/Cluster/MembershipGossipTests.swift b/Tests/DistributedActorsTests/Cluster/MembershipGossipTests.swift index 9fcc66f61..0e06cfb3b 100644 --- a/Tests/DistributedActorsTests/Cluster/MembershipGossipTests.swift +++ b/Tests/DistributedActorsTests/Cluster/MembershipGossipTests.swift @@ -271,11 +271,7 @@ final class MembershipGossipTests: XCTestCase { """, owner: self.nodeB, nodes: self.allNodes ) - pprint("membership = \(gossip)") - pprint("ahead = \(concurrent)") - let directive = gossip.mergeForward(incoming: concurrent) - pprint("membership.version = \(gossip.version)") gossip.owner.shouldEqual(self.nodeA) directive.effectiveChanges.count.shouldEqual(0) diff --git a/Tests/DistributedActorsTests/Cluster/ProtobufRoundTripTests.swift b/Tests/DistributedActorsTests/Cluster/ProtobufRoundTripTests.swift index 811cadcd1..efc6e7405 100644 --- a/Tests/DistributedActorsTests/Cluster/ProtobufRoundTripTests.swift +++ b/Tests/DistributedActorsTests/Cluster/ProtobufRoundTripTests.swift @@ -56,7 +56,7 @@ final class ProtobufRoundTripTests: XCTestCase { func test_roundTrip_Wire_HandshakeOffer() throws { try self.check( - Wire.HandshakeOffer(version: .init(reserved: 2, major: 3, minor: 5, patch: 5), from: self.node, to: self.node.node), + Wire.HandshakeOffer(version: .init(reserved: 2, major: 3, minor: 5, patch: 5), originNode: self.node, targetNode: self.node.node), toProto: ProtoHandshakeOffer.init, fromProto: Wire.HandshakeOffer.init ) diff --git a/Tests/DistributedActorsTests/Cluster/RemotingHandshakeStateMachineTests.swift b/Tests/DistributedActorsTests/Cluster/RemotingHandshakeStateMachineTests.swift index a1a912a5e..8fe288bff 100644 --- a/Tests/DistributedActorsTests/Cluster/RemotingHandshakeStateMachineTests.swift +++ b/Tests/DistributedActorsTests/Cluster/RemotingHandshakeStateMachineTests.swift @@ -29,19 +29,20 @@ final class RemoteHandshakeStateMachineTests: XCTestCase { func test_handshake_happyPath() throws { let serverKernel = ClusterShellState.makeTestMock(side: .server) - let serverAddress = serverKernel.myselfNode + let serverAddress = serverKernel.localNode let clientKernel = ClusterShellState.makeTestMock(side: .client) { settings in settings.node.port = 2222 } // client - let clientInitiated = HSM.InitiatedState(settings: clientKernel.settings, localNode: clientKernel.myselfNode, connectTo: serverAddress.node, whenCompleted: nil) + let clientInitiated = HSM.InitiatedState(settings: clientKernel.settings, localNode: clientKernel.localNode, connectTo: serverAddress.node, whenCompleted: nil) let offer = clientInitiated.makeOffer() // server let received = HSM.HandshakeReceivedState(state: serverKernel, offer: offer, whenCompleted: nil) // TODO: test that it completes? - _ = received._acceptAndMakeCompletedState() // TODO: hide this +// _ = received._acceptAndMakeCompletedState() // TODO: hide this +// received.whenCompleted. let serverCompleted: HSM.CompletedState switch received.negotiate() { @@ -56,11 +57,11 @@ final class RemoteHandshakeStateMachineTests: XCTestCase { // then - serverCompleted.localNode.shouldEqual(serverKernel.myselfNode) - serverCompleted.remoteNode.shouldEqual(clientKernel.myselfNode) + serverCompleted.localNode.shouldEqual(serverKernel.localNode) + serverCompleted.remoteNode.shouldEqual(clientKernel.localNode) - clientCompleted.remoteNode.shouldEqual(serverKernel.myselfNode) - clientCompleted.localNode.shouldEqual(clientKernel.myselfNode) + clientCompleted.remoteNode.shouldEqual(serverKernel.localNode) + clientCompleted.localNode.shouldEqual(clientKernel.localNode) } // ==== ------------------------------------------------------------------------------------------------------------ @@ -68,14 +69,14 @@ final class RemoteHandshakeStateMachineTests: XCTestCase { func test_negotiate_server_shouldAcceptClient_newerPatch() throws { let serverKernel = ClusterShellState.makeTestMock(side: .server) - let serverAddress = serverKernel.myselfNode + let serverAddress = serverKernel.localNode let clientKernel = ClusterShellState.makeTestMock(side: .client) { settings in settings.node.port = 2222 settings._protocolVersion.patch += 1 } - let clientInitiated = HSM.InitiatedState(settings: clientKernel.settings, localNode: clientKernel.myselfNode, connectTo: serverAddress.node, whenCompleted: nil) + let clientInitiated = HSM.InitiatedState(settings: clientKernel.settings, localNode: clientKernel.localNode, connectTo: serverAddress.node, whenCompleted: nil) let offer = clientInitiated.makeOffer() // server @@ -93,14 +94,14 @@ final class RemoteHandshakeStateMachineTests: XCTestCase { func test_negotiate_server_shouldRejectClient_newerMajor() throws { let serverKernel = ClusterShellState.makeTestMock(side: .server) - let serverAddress = serverKernel.myselfNode + let serverAddress = serverKernel.localNode let clientKernel = ClusterShellState.makeTestMock(side: .client) { settings in settings.node.port = 2222 settings._protocolVersion.major += 1 } - let clientInitiated = HSM.InitiatedState(settings: clientKernel.settings, localNode: clientKernel.myselfNode, connectTo: serverAddress.node, whenCompleted: nil) + let clientInitiated = HSM.InitiatedState(settings: clientKernel.settings, localNode: clientKernel.localNode, connectTo: serverAddress.node, whenCompleted: nil) let offer = clientInitiated.makeOffer() // server @@ -124,14 +125,14 @@ final class RemoteHandshakeStateMachineTests: XCTestCase { func test_onTimeout_shouldReturnNewHandshakeOffersMultipleTimes() throws { let serverKernel = ClusterShellState.makeTestMock(side: .server) - let serverAddress = serverKernel.myselfNode + let serverAddress = serverKernel.localNode let clientKernel = ClusterShellState.makeTestMock(side: .client) { settings in settings.node.port = 8228 } // client - var clientInitiated = HSM.InitiatedState(settings: clientKernel.settings, localNode: clientKernel.myselfNode, connectTo: serverAddress.node, whenCompleted: nil) + var clientInitiated = HSM.InitiatedState(settings: clientKernel.settings, localNode: clientKernel.localNode, connectTo: serverAddress.node, whenCompleted: nil) guard case .scheduleRetryHandshake = clientInitiated.onHandshakeTimeout() else { throw shouldNotHappen("Expected retry attempt after handshake timeout") diff --git a/Tests/DistributedActorsTests/ParentChildActorTests.swift b/Tests/DistributedActorsTests/ParentChildActorTests.swift index c83b53781..772147153 100644 --- a/Tests/DistributedActorsTests/ParentChildActorTests.swift +++ b/Tests/DistributedActorsTests/ParentChildActorTests.swift @@ -141,7 +141,6 @@ final class ParentChildActorTests: ActorSystemTestBase { parent.tell(.spawnChild(name: "kid", behavior: self.childBehavior(probe: p.ref))) guard case .spawned(let child) = try p.expectMessage() else { throw p.error() } - pnote("Hello: \(child)") let unknownName = "capybara" parent.tell(.findByName(name: unknownName)) @@ -167,7 +166,6 @@ final class ParentChildActorTests: ActorSystemTestBase { parent.tell(.spawnAnonymousChild(behavior: self.childBehavior(probe: p.ref))) guard case .spawned(let child) = try p.expectMessage() else { throw p.error() } - pnote("Hello: \(child)") parent.tell(.findByName(name: child.address.name)) try p.expectMessage(.childFound(name: child.address.name, ref: child)) // should return same (or equal) ref From f3e5f72593c5e0bf11d4304940b9e19f1c6ac096 Mon Sep 17 00:00:00 2001 From: Konrad `ktoso` Malawski Date: Thu, 30 Apr 2020 21:28:37 +0900 Subject: [PATCH 11/18] =test adjust that we use 127.0.0.1 in tests now --- .../CRDT/Protobuf/CRDT+SerializationTests.swift | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Tests/DistributedActorsTests/CRDT/Protobuf/CRDT+SerializationTests.swift b/Tests/DistributedActorsTests/CRDT/Protobuf/CRDT+SerializationTests.swift index 59e6c6e63..206c022e5 100644 --- a/Tests/DistributedActorsTests/CRDT/Protobuf/CRDT+SerializationTests.swift +++ b/Tests/DistributedActorsTests/CRDT/Protobuf/CRDT+SerializationTests.swift @@ -232,7 +232,7 @@ final class CRDTSerializationTests: ActorSystemTestBase { let serialized = try system.serialization.serialize(map) let deserialized = try system.serialization.deserialize(as: CRDT.ORMap>.self, from: serialized) - "\(deserialized.replicaID)".shouldContain("actor:sact://CRDTSerializationTests@localhost:9001/user/alpha") + "\(deserialized.replicaID)".shouldContain("actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/alpha") deserialized.defaultValue.shouldBeNil() deserialized._keys.elements.shouldEqual(["s1", "s2"]) deserialized._values.count.shouldEqual(2) From 639cc2a5ae8de5a2290d4a381658d75a45070837 Mon Sep 17 00:00:00 2001 From: Konrad `ktoso` Malawski Date: Thu, 30 Apr 2020 22:12:43 +0900 Subject: [PATCH 12/18] =test,downing increase timeouts as current default detection gives quite some leeway --- .../Cluster/DowningStrategy/DowningClusteredTests.swift | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/Tests/DistributedActorsTests/Cluster/DowningStrategy/DowningClusteredTests.swift b/Tests/DistributedActorsTests/Cluster/DowningStrategy/DowningClusteredTests.swift index 61d5b999a..d2d99809b 100644 --- a/Tests/DistributedActorsTests/Cluster/DowningStrategy/DowningClusteredTests.swift +++ b/Tests/DistributedActorsTests/Cluster/DowningStrategy/DowningClusteredTests.swift @@ -107,8 +107,9 @@ final class DowningClusteredTests: ClusteredNodesTestBase { } // collect all events regarding the expectedDownNode's membership lifecycle - let eventsOnOther = try eventsProbeOther.fishFor(Cluster.MembershipChange.self, within: .seconds(20), expectedDownMemberEventsFishing(on: otherNotDownPairSystem)) - let eventsOnThird = try eventsProbeThird.fishFor(Cluster.MembershipChange.self, within: .seconds(20), expectedDownMemberEventsFishing(on: thirdNeverDownSystem)) + // (the timeout is fairly large here to tolerate slow CI and variations how the events get propagated, normally they propagate quite quickly) + let eventsOnOther = try eventsProbeOther.fishFor(Cluster.MembershipChange.self, within: .seconds(30), expectedDownMemberEventsFishing(on: otherNotDownPairSystem)) + let eventsOnThird = try eventsProbeThird.fishFor(Cluster.MembershipChange.self, within: .seconds(30), expectedDownMemberEventsFishing(on: thirdNeverDownSystem)) eventsOnOther.shouldContain(where: { change in change.toStatus.isDown && (change.fromStatus == .joining || change.fromStatus == .up) }) eventsOnOther.shouldContain(Cluster.MembershipChange(node: expectedDownNode, fromStatus: .down, toStatus: .removed)) From 805fec78e05ca7df45f8ced810069c5e46a847c8 Mon Sep 17 00:00:00 2001 From: Konrad `ktoso` Malawski Date: Thu, 30 Apr 2020 22:32:11 +0900 Subject: [PATCH 13/18] =test increase default expectation timeout; not much pentalty for quick msgs --- .../Cluster/ClusterShell.swift | 12 +------ .../Cluster/ClusterShellState.swift | 13 ++++---- .../Cluster/HandshakeStateMachine.swift | 7 ----- .../Cluster/SWIM/SWIMShell.swift | 31 +------------------ .../ActorTestKit.swift | 6 ++-- .../ClusterLeaderActionsClusteredTests.swift | 3 ++ .../RemotingHandshakeStateMachineTests.swift | 2 -- 7 files changed, 15 insertions(+), 59 deletions(-) diff --git a/Sources/DistributedActors/Cluster/ClusterShell.swift b/Sources/DistributedActors/Cluster/ClusterShell.swift index c7fb8578b..56180cb84 100644 --- a/Sources/DistributedActors/Cluster/ClusterShell.swift +++ b/Sources/DistributedActors/Cluster/ClusterShell.swift @@ -676,16 +676,6 @@ extension ClusterShell { func connectSendHandshakeOffer(_ context: ActorContext, _ state: ClusterShellState, initiated: HandshakeStateMachine.InitiatedState) -> Behavior { var state = state - -// if let existingAssociation = self.getExistingAssociation(with: initiated.remoteNode) { -// state.log.info("No need to extend handshake offer, node [\(existingAssociation.remoteNode)] is already associated.", metadata: [ -// "handshake": "\(existingAssociation.state)" -// ]) -// existingAssociation.whenComplete(initiated.whenCompleted) -// // TODO: ensure state has it as well; clear the handshake -// return self.ready(state: state) -// } - state.log.info("Extending handshake offer to \(initiated.remoteNode))") // TODO: log retry stats? let offer: Wire.HandshakeOffer = initiated.makeOffer() @@ -1152,8 +1142,8 @@ extension ClusterShell { // ==== ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ // Terminate association and Down the (other) node - self.terminateAssociation(context.system, state: &state, memberToDown.node) state = self.interpretLeaderActions(context.system, state, state.collectLeaderActions()) + self.terminateAssociation(context.system, state: &state, memberToDown.node) return state } } diff --git a/Sources/DistributedActors/Cluster/ClusterShellState.swift b/Sources/DistributedActors/Cluster/ClusterShellState.swift index f3d58303f..8e525de71 100644 --- a/Sources/DistributedActors/Cluster/ClusterShellState.swift +++ b/Sources/DistributedActors/Cluster/ClusterShellState.swift @@ -180,15 +180,14 @@ extension ClusterShellState { /// /// - Faults: when called in wrong state of an ongoing handshake /// - Returns: if present, the (now removed) handshake state that was aborted, hil otherwise. - // THIS SIGNATURE IS SCARY; dont kill a Node, kill a specific channel (!) - mutating func closeOutboundHandshakeChannel(with node: Node, file: String = #file, line: UInt = #line) -> HandshakeStateMachine.State? { + mutating func closeOutboundHandshakeChannel(with node: Node) -> HandshakeStateMachine.State? { guard let state = self._handshakes.removeValue(forKey: node) else { return nil } switch state { case .initiated(let initiated): if let channel = initiated.channel { - self.log.warning("ABORTING OUTBOUND handshake channel: \(channel) [AT: \(file):\(line)]") + self.log.trace("Aborting OUTBOUND handshake channel: \(channel)") channel.close().whenFailure { [log = self.log] error in log.warning("Failed to abortOutgoingHandshake (close) channel [\(channel)], error: \(error)") @@ -211,7 +210,7 @@ extension ClusterShellState { /// /// - Returns: if present, the (now removed) handshake state that was aborted, hil otherwise. mutating func closeHandshakeChannel(offer: Wire.HandshakeOffer, channel: Channel) { - self.log.warning("ABORTING INBOUND handshake channel: \(channel)") + self.log.trace("Aborting INBOUND handshake channel: \(channel)") channel.close().whenFailure { [log = self.log, metadata = self.metadata] error in log.warning("Failed to abortIncomingHandshake (close) channel [\(channel)], error: \(error)", metadata: metadata) @@ -304,8 +303,10 @@ extension ClusterShellState { mutating func incomingHandshakeAccept(_ accept: Wire.HandshakeAccept) -> HandshakeStateMachine.CompletedState? { guard let inProgressHandshake = self._handshakes[accept.targetNode.node] else { - // TODO: what if node that sent handshake, has already terminated -- would we have removed the in progress handshake already causing this? - // fatalError("Accept incoming [\(accept)] for handshake which was not in progress! On node: \(self.myselfNode), cluster shell state: \(self), membership: \(self.membership)") // TODO: model differently + self.log.warning("Attempted to accept incoming [\(accept)] for handshake which was not in progress!", metadata: [ + "clusterShell": "\(self)", + "membership": "\(self.membership)", + ]) return nil } diff --git a/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift b/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift index 21798c747..ba829e3ab 100644 --- a/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift +++ b/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift @@ -169,13 +169,6 @@ internal struct HandshakeStateMachine { self.whenCompleted = whenCompleted } -// // do not call directly, rather obtain the completed state via negotiate() -// func _acceptAndMakeCompletedState() -> CompletedState { -// let completed = CompletedState(fromReceived: self, remoteNode: offer.originNode) -// self.whenCompleted?.succeed(.accept(completed.makeAccept())) -// return completed -// } - func negotiate() -> HandshakeStateMachine.NegotiateDirective { guard self.boundAddress.node == self.offer.targetNode else { let error = HandshakeError.targetHandshakeAddressMismatch(self.offer, selfNode: self.boundAddress) diff --git a/Sources/DistributedActors/Cluster/SWIM/SWIMShell.swift b/Sources/DistributedActors/Cluster/SWIM/SWIMShell.swift index 9038205d2..960973116 100644 --- a/Sources/DistributedActors/Cluster/SWIM/SWIMShell.swift +++ b/Sources/DistributedActors/Cluster/SWIM/SWIMShell.swift @@ -512,7 +512,7 @@ internal struct SWIMShell { self.clusterRef.tell(.command(.failureDetectorReachabilityChanged(change.member.node, reachability))) } - // TODO: remove this + // TODO: remove or simplify; SWIM/Associations: Simplify/remove withEnsuredAssociation #601 /// Use to ensure an association to given remote node exists. func withEnsuredAssociation(_ context: ActorContext, remoteNode: UniqueNode?, continueWithAssociation: @escaping (Result) -> Void) { // this is a local node, so we don't need to connect first @@ -523,35 +523,6 @@ internal struct SWIMShell { // actor refs handle kicking off associations automatically when attempting to send to them; so we do nothing here (!!!) continueWithAssociation(.success(remoteNode)) - -// guard let clusterShell = context.system._cluster else { -// continueWithAssociation(.failure(EnsureAssociationError("ClusterShell not available when trying to ensure associated with: \(reflecting: remoteNode)"))) -// return -// } -// -// let associationState = clusterShell.getEnsureAssociation(with: remoteNode) -// switch associationState { -// case .association(let control): -// continueWithAssociation(.success(control.remoteNode)) -// case .tombstone: -// let msg = "Association target node is already .tombstoned, not associating. Node \(reflecting: remoteNode) likely to be removed from gossip shortly." -// continueWithAssociation(.failure(EnsureAssociationError(msg))) -// return // we shall not associate with this tombstoned node (!) -// } -// -// // ensure connection to new node ~~~ -// let ref = context.messageAdapter(from: ClusterShell.HandshakeResult.self) { (result: ClusterShell.HandshakeResult) in -// switch result { -// case .success(let uniqueNode): -// return SWIM.Message.local(.monitor(uniqueNode)) -// case .failure(let error): -// context.log.debug("Did not associate with \(reflecting: remoteNode), reason: \(error)") -// return nil // drop the message -// } -// } -// -// context.log.trace("Requesting handshake with \(remoteNode.node)") -// self.clusterRef.tell(.command(.handshakeWith(remoteNode.node, replyTo: ref))) } struct EnsureAssociationError: Error { diff --git a/Sources/DistributedActorsTestKit/ActorTestKit.swift b/Sources/DistributedActorsTestKit/ActorTestKit.swift index be4a98a78..89c26c295 100644 --- a/Sources/DistributedActorsTestKit/ActorTestKit.swift +++ b/Sources/DistributedActorsTestKit/ActorTestKit.swift @@ -57,7 +57,7 @@ public struct TestError: Error, Hashable { public struct ActorTestKitSettings { /// Timeout used by default by all the `expect...` and `within` functions defined on the testkit and test probes. - var expectationTimeout: TimeAmount = .seconds(3) + var expectationTimeout: TimeAmount = .seconds(5) } // ==== ---------------------------------------------------------------------------------------------------------------- @@ -292,7 +292,7 @@ extension ActorTestKit { /// If unable to resolve a not-dead reference, this function throws, rather than returning the dead reference. /// /// This is useful when the resolution might be racing against the startup of the actor we are trying to resolve. - public func _eventuallyResolve(address: ActorAddress, of: Message.Type = Message.self, within: TimeAmount = .seconds(3)) throws -> ActorRef { + public func _eventuallyResolve(address: ActorAddress, of: Message.Type = Message.self, within: TimeAmount = .seconds(5)) throws -> ActorRef { let context = ResolveContext(address: address, system: self.system) return try self.eventually(within: .seconds(3)) { @@ -424,7 +424,7 @@ extension ActorTestKit { /// /// Examples: /// - /// testKit.eventually(within: .seconds(1)) { + /// testKit.eventually(within: .seconds(3)) { /// guard ... else { throw testKit.error("failed to extract expected information") } /// } public func error(_ message: String? = nil, file: StaticString = #file, line: UInt = #line, column: UInt = #column) -> Error { diff --git a/Tests/DistributedActorsTests/Cluster/ClusterLeaderActionsClusteredTests.swift b/Tests/DistributedActorsTests/Cluster/ClusterLeaderActionsClusteredTests.swift index 3394867a1..5427dd476 100644 --- a/Tests/DistributedActorsTests/Cluster/ClusterLeaderActionsClusteredTests.swift +++ b/Tests/DistributedActorsTests/Cluster/ClusterLeaderActionsClusteredTests.swift @@ -241,6 +241,9 @@ final class ClusterLeaderActionsClusteredTests: ClusteredNodesTestBase { // on the leader node, the other node noticed as up: let eventsOnFirstSub = try p1.expectMessages(count: 9) + for event in eventsOnFirstSub { + pinfo("Captured event: \(event)") + } eventsOnFirstSub.shouldContain(.snapshot(.empty)) eventsOnFirstSub.shouldContain(.membershipChange(.init(node: first.cluster.node, fromStatus: nil, toStatus: .joining))) eventsOnFirstSub.shouldContain(.membershipChange(.init(node: secondNode, fromStatus: nil, toStatus: .joining))) diff --git a/Tests/DistributedActorsTests/Cluster/RemotingHandshakeStateMachineTests.swift b/Tests/DistributedActorsTests/Cluster/RemotingHandshakeStateMachineTests.swift index 8fe288bff..68242874a 100644 --- a/Tests/DistributedActorsTests/Cluster/RemotingHandshakeStateMachineTests.swift +++ b/Tests/DistributedActorsTests/Cluster/RemotingHandshakeStateMachineTests.swift @@ -41,8 +41,6 @@ final class RemoteHandshakeStateMachineTests: XCTestCase { // server let received = HSM.HandshakeReceivedState(state: serverKernel, offer: offer, whenCompleted: nil) // TODO: test that it completes? -// _ = received._acceptAndMakeCompletedState() // TODO: hide this -// received.whenCompleted. let serverCompleted: HSM.CompletedState switch received.negotiate() { From 78b3d4c379845ba61e81d88c92d9c1f9a8a15711 Mon Sep 17 00:00:00 2001 From: Konrad `ktoso` Malawski Date: Fri, 1 May 2020 15:29:59 +0900 Subject: [PATCH 14/18] wip fixing associations more and more --- .../CRDT/CRDT+ReplicatorShell.swift | 14 +-- .../Cluster/Association.swift | 5 +- .../Cluster/Cluster+Event.swift | 26 +++--- .../Cluster/Cluster+Gossip.swift | 11 +++ .../Cluster/Cluster+Membership.swift | 7 +- .../Cluster/ClusterEventStream.swift | 1 + .../Cluster/ClusterShell.swift | 87 ++++++++++++++----- .../Cluster/ClusterShellState.swift | 2 +- .../Cluster/Downing/DowningStrategy.swift | 2 +- .../Cluster/HandshakeStateMachine.swift | 5 -- .../Cluster/SWIM/SWIMSettings.swift | 2 +- .../Transport/TransportPipelines.swift | 18 ++-- .../Protobuf/WireProtocol+Serialization.swift | 2 +- .../DistributedActors/String+Extensions.swift | 12 ++- .../Cluster/ClusteredNodesTestBase.swift | 2 +- .../Cluster/AssociationClusteredTests.swift | 65 +++++++------- .../MembershipTests.swift | 15 ++++ 17 files changed, 174 insertions(+), 102 deletions(-) diff --git a/Sources/DistributedActors/CRDT/CRDT+ReplicatorShell.swift b/Sources/DistributedActors/CRDT/CRDT+ReplicatorShell.swift index 5c0c2212c..4ebeee66d 100644 --- a/Sources/DistributedActors/CRDT/CRDT+ReplicatorShell.swift +++ b/Sources/DistributedActors/CRDT/CRDT+ReplicatorShell.swift @@ -84,14 +84,14 @@ extension CRDT.Replicator { switch event { case .membershipChange(let change) where change.toStatus == .up: let member = change.member - if member.node != context.system.cluster.node { // exclude this (local) node - self.tracelog(context, .addMember, message: member) - let remoteReplicatorRef = makeReplicatorRef(member.node) - self.remoteReplicators.insert(remoteReplicatorRef) - } else { - context.log.trace("Skip adding member \(member) to replicator because it is the same as local node", metadata: self.metadata(context)) + guard member.node != context.system.cluster.node else { + return // Skip adding member to replicator because it is the same as local node } + self.tracelog(context, .addMember, message: member) + let remoteReplicatorRef = makeReplicatorRef(member.node) + self.remoteReplicators.insert(remoteReplicatorRef) + case .membershipChange(let change) where change.toStatus >= .down: let member = change.member self.tracelog(context, .removeMember, message: member) @@ -106,7 +106,7 @@ extension CRDT.Replicator { case .membershipChange: context.log.trace("Ignoring cluster event \(event), only interested in >= .up events", metadata: self.metadata(context)) default: - () // ignore other events + return // ignore other events } } diff --git a/Sources/DistributedActors/Cluster/Association.swift b/Sources/DistributedActors/Cluster/Association.swift index c814c4376..96d919b36 100644 --- a/Sources/DistributedActors/Cluster/Association.swift +++ b/Sources/DistributedActors/Cluster/Association.swift @@ -156,7 +156,7 @@ final class Association: CustomStringConvertible { } var description: String { - "AssociatedState(\(self.state), selfNode: \(self.selfNode), remoteNode: \(self.remoteNode))" + "AssociatedState(\(self.state), selfNode: \(reflecting: self.selfNode), remoteNode: \(reflecting: self.remoteNode))" } } @@ -179,10 +179,13 @@ extension Association { self.lock.withLockVoid { switch self.state { case .associating(let sendQueue): + pprint("SEND ENQUEUE = \(envelope)") sendQueue.enqueue(envelope) case .associated(let channel): + pprint("SEND NOW = \(envelope)") channel.writeAndFlush(envelope, promise: promise) case .tombstone(let deadLetters): + pprint("SEND DEAD = \(envelope)") deadLetters.tell(.init(envelope.underlyingMessage, recipient: envelope.recipient)) } } diff --git a/Sources/DistributedActors/Cluster/Cluster+Event.swift b/Sources/DistributedActors/Cluster/Cluster+Event.swift index 0f78ce0c2..7bcac0cfd 100644 --- a/Sources/DistributedActors/Cluster/Cluster+Event.swift +++ b/Sources/DistributedActors/Cluster/Cluster+Event.swift @@ -34,7 +34,7 @@ extension Cluster { /// or may originate from local decisions (such as joining or downing). public struct MembershipChange: Hashable { /// Current member that is part of the membership after this change - public internal(set) var member: Cluster.Member + public internal(set) var member: Member /// The node which the change concerns. public var node: UniqueNode { @@ -43,20 +43,20 @@ extension Cluster { /// Only set if the change is a "replacement", which can happen only if a node joins /// from the same physical address (host + port), however its UID has changed. - internal private(set) var replaced: Cluster.Member? + internal private(set) var replaced: Member? /// A replacement means that a new node appeared on the same host/port, and thus the old node must be assumed down. - internal var replacementDownPreviousNodeChange: Cluster.MembershipChange? { + internal var replacementDownPreviousNodeChange: MembershipChange? { guard let replacedMember = self.replaced else { return nil } return .init(member: replacedMember, toStatus: .down) } - public internal(set) var fromStatus: Cluster.MemberStatus? - public let toStatus: Cluster.MemberStatus + public internal(set) var fromStatus: MemberStatus? + public let toStatus: MemberStatus - init(member: Cluster.Member, toStatus: Cluster.MemberStatus? = nil) { + init(member: Member, toStatus: MemberStatus? = nil) { // FIXME: enable these assertions // assertBacktrace( // toStatus == nil || !(toStatus == .removed && member.status != .down), @@ -81,7 +81,7 @@ extension Cluster { } } - init(node: UniqueNode, fromStatus: Cluster.MemberStatus?, toStatus: Cluster.MemberStatus) { + init(node: UniqueNode, fromStatus: MemberStatus?, toStatus: MemberStatus) { // FIXME: enable these assertions // assertBacktrace( // !(toStatus == .removed && fromStatus != .down), @@ -97,7 +97,7 @@ extension Cluster { } /// Use to create a "replacement", when the previousNode and node are different (i.e. they should only differ in ID, not host/port) - init(replaced: Cluster.Member, by newMember: Cluster.Member) { + init(replaced: Member, by newMember: Member) { assert(replaced.node.host == newMember.node.host, "Replacement Cluster.MembershipChange should be for same non-unique node; Was: \(replaced), and \(newMember)") assert(replaced.node.port == newMember.node.port, "Replacement Cluster.MembershipChange should be for same non-unique node; Was: \(replaced), and \(newMember)") @@ -144,17 +144,13 @@ extension Cluster.MembershipChange { } } -extension Cluster.MembershipChange: CustomStringConvertible, CustomDebugStringConvertible { +extension Cluster.MembershipChange: CustomStringConvertible { public var description: String { - "Cluster.MembershipChange(node: \(node), replaced: \(replaced, orElse: "nil"), fromStatus: \(fromStatus.map { "\($0)" } ?? "nil"), toStatus: \(toStatus))" - } - - public var debugDescription: String { let base: String if let replaced = self.replaced { - base = "[replaced:\(String(reflecting: replaced))] by \(reflecting: self.node)" + base = "[replaced:\(reflecting: replaced)] by \(reflecting: self.node)" } else { - base = "\(self.node)" + base = "\(reflecting: self.node)" } return base + " :: " + diff --git a/Sources/DistributedActors/Cluster/Cluster+Gossip.swift b/Sources/DistributedActors/Cluster/Cluster+Gossip.swift index bfa091e8f..81576ec16 100644 --- a/Sources/DistributedActors/Cluster/Cluster+Gossip.swift +++ b/Sources/DistributedActors/Cluster/Cluster+Gossip.swift @@ -25,6 +25,17 @@ extension Cluster { /// Each row in the table represents what versionVector we know the given node has observed recently. /// It may have in the mean time of course observed a new version already. // TODO: There is tons of compression opportunity about not having to send full tables around in general, but for now we will just send them around + // FIXME: ensure that we never have a seen entry for a non-member + // bad: "actor/message": Gossip( + // owner: sact://first:2342486320@127.0.0.1:9001, + // seen: Cluster.Gossip.SeenTable( + // [sact://second:4264003847@127.0.0.1:9002: [uniqueNode:sact://second@127.0.0.1:9002: 2], + // sact://first:2342486320@127.0.0.1:9001: [uniqueNode:sact://first@127.0.0.1:9001: 4, uniqueNode:sact://second@127.0.0.1:9002: 2]] + // ), + // membership: Membership(count: 2, leader: Member(sact://first@127.0.0.1:9001, status: joining, reachability: reachable), + // members: [ + // Member(sact://first:2342486320@127.0.0.1:9001, status: joining, reachability: reachable), + // Member(sact://second-REPLACEMENT:871659343@127.0.0.1:9002, status: joining, reachability: reachable)])) var seen: Cluster.Gossip.SeenTable /// The version vector of this gossip and the `Membership` state owned by it. var version: VersionVector { diff --git a/Sources/DistributedActors/Cluster/Cluster+Membership.swift b/Sources/DistributedActors/Cluster/Cluster+Membership.swift index 4b2612fec..0562aa437 100644 --- a/Sources/DistributedActors/Cluster/Cluster+Membership.swift +++ b/Sources/DistributedActors/Cluster/Cluster+Membership.swift @@ -230,11 +230,10 @@ extension Cluster.Membership: Hashable { } } -extension Cluster.Membership: CustomStringConvertible, CustomDebugStringConvertible { +extension Cluster.Membership: CustomStringConvertible, CustomDebugStringConvertible, CustomPrettyStringConvertible { /// Pretty multi-line output of a membership, useful for manual inspection - public func prettyDescription(label: String) -> String { - var res = "Membership \(label):" - res += "\n LEADER: \(self.leader, orElse: ".none")" + var prettyDescription: String { + var res = "LEADER: \(self.leader, orElse: ".none")" for member in self._members.values.sorted(by: { $0.node.node.port < $1.node.node.port }) { res += "\n \(reflecting: member.node) STATUS: [\(member.status.rawValue, leftPadTo: Cluster.MemberStatus.maxStrLen)]" } diff --git a/Sources/DistributedActors/Cluster/ClusterEventStream.swift b/Sources/DistributedActors/Cluster/ClusterEventStream.swift index 3261c8ade..13d602259 100644 --- a/Sources/DistributedActors/Cluster/ClusterEventStream.swift +++ b/Sources/DistributedActors/Cluster/ClusterEventStream.swift @@ -60,6 +60,7 @@ internal enum ClusterEventStream { context.log.trace( "Published event \(event) to \(subscribers.count) subscribers", metadata: [ + "eventStream/event": "\(reflecting: event)", "eventStream/subscribers": Logger.MetadataValue.array(subscribers.map { Logger.MetadataValue.stringConvertible($0.key) }), diff --git a/Sources/DistributedActors/Cluster/ClusterShell.swift b/Sources/DistributedActors/Cluster/ClusterShell.swift index 56180cb84..60a96b022 100644 --- a/Sources/DistributedActors/Cluster/ClusterShell.swift +++ b/Sources/DistributedActors/Cluster/ClusterShell.swift @@ -51,7 +51,7 @@ internal class ClusterShell { internal func getAnyExistingAssociation(with node: Node) -> Association? { self._associationsLock.withLock { - // TODO: a bit terrible; perhaps we should key be Node and then confirm by UniqueNode? + // TODO: a bit terrible; perhaps key should be Node and then confirm by UniqueNode? // this used to be separated in the State keeping them by Node and here we kept by unique though that caused other challenges self._associations.first { key, _ in key.node == node @@ -124,23 +124,47 @@ internal class ClusterShell { } guard let removedAssociation = removedAssociationOption else { - system.log.warning("Attempted to terminate non-existing association [\(remoteNode)].") + system.log.warning("Attempted to terminate non-existing association [\(reflecting: remoteNode)].") return } + system.log.warning("Terminate existing association [\(reflecting: remoteNode)].") + // notify the failure detector, that we shall assume this node as dead from here on. // it's gossip will also propagate the information through the cluster traceLog_Remote(system.cluster.node, "Finish terminate association [\(remoteNode)]: Notifying SWIM, .confirmDead") self._swimRef?.tell(.local(.confirmDead(remoteNode))) - // Ensure to remove (completely) the member from the Membership, it is not even .leaving anymore. - if state.membership.mark(remoteNode, as: .down) == nil { - // it was already removed, nothing to do - state.log.trace( - "Finish association with \(remoteNode), yet node not in membership already?", - metadata: ["cluster/membership": "\(state.membership)"] - ) - } // else: Note that we CANNOT remove() just yet, as we only want to do this when all nodes have seen the down/leaving + let before = state.membership + // it is important that we first check the contains; as otherwise we'd re-add a .down member for what was already removed (!) + if state.membership.contains(remoteNode) { + // Ensure to remove (completely) the member from the Membership, it is not even .leaving anymore. + if state.membership.mark(remoteNode, as: .down) == nil { + // it was already removed, nothing to do + state.log.trace( + "Terminate association with \(reflecting: remoteNode), yet node not in membership already?", metadata: [ + "cluster/membership": "\(pretty: state.membership)" + ] + ) + } // else: Note that we CANNOT remove() just yet, as we only want to do this when all nodes have seen the down/leaving + } + +// state.membership.mark(remoteNode, as: .down); remoteNode = sact://second:1026573596@127.0.0.1:9002 +// BEFORE MARK DOWN = LEADER: Member(sact://first@127.0.0.1:9001, status: joining, reachability: reachable) +// sact://first:1954943626@127.0.0.1:9001 STATUS: [joining] +// sact://second-REPLACEMENT:3526768720@127.0.0.1:9002 STATUS: [joining] +// +// AFTER MARK DOWN = LEADER: Member(sact://first@127.0.0.1:9001, status: joining, reachability: reachable) +// sact://first:1954943626@127.0.0.1:9001 STATUS: [joining] +// sact://second-REPLACEMENT:3526768720@127.0.0.1:9002 STATUS: [ down] +// sact://second:1026573596@127.0.0.1:9002 STATUS: [ down] + + + pprint(""" + state.membership.mark(remoteNode, as: .down); remoteNode = \(reflecting: remoteNode) + BEFORE MARK DOWN = \(pretty: before) + AFTER MARK DOWN = \(pretty: state.membership) + """) // The last thing we attempt to do with the other node is to shoot it, // in case it's a "zombie" that still may receive messages for some reason. @@ -435,7 +459,7 @@ extension ClusterShell { case .handshakeWith(let node, let replyTo): return self.beginHandshake(context, state, with: node, replyTo: replyTo) case .retryHandshake(let initiated): - return self.connectSendHandshakeOffer(context, state, initiated: initiated) + return self.retryHandshake(context, state, initiated: initiated) case .failureDetectorReachabilityChanged(let node, let reachability): guard let member = state.membership.uniqueMember(node) else { @@ -673,6 +697,14 @@ extension ClusterShell { return self.ready(state: state) } } + internal func retryHandshake(_ context: ActorContext, _ state: ClusterShellState, initiated: HandshakeStateMachine.InitiatedState) -> Behavior { + state.log.info("Retry handshake with: \(initiated.remoteNode)") + + // TODO: update retry counter, perhaps give up + + return self.connectSendHandshakeOffer(context, state, initiated: initiated) + + } func connectSendHandshakeOffer(_ context: ActorContext, _ state: ClusterShellState, initiated: HandshakeStateMachine.InitiatedState) -> Behavior { var state = state @@ -722,25 +754,33 @@ extension ClusterShell { case .negotiateIncoming(let hsm): // handshake is allowed to proceed switch hsm.negotiate() { - case .acceptAndAssociate(let completedHandshake): + case .acceptAndAssociate(let handshakeCompleted): state.log.info("Accept handshake with \(reflecting: offer.originNode)!", metadata: [ "handshake/channel": "\(inboundChannel)", ]) // accept handshake and store completed association - let directive = state.completeHandshakeAssociate(self, completedHandshake, channel: inboundChannel) + let directive = state.completeHandshakeAssociate(self, handshakeCompleted, channel: inboundChannel) - // send accept to other node - let accept = completedHandshake.makeAccept(whenHandshakeReplySent: { () in + // prepare accept + let accept = handshakeCompleted.makeAccept(whenHandshakeReplySent: { () in self.completeAssociation(directive) + state.log.debug("Associated with: \(reflecting: handshakeCompleted.remoteNode)", metadata: [ + "membership/change": "\(directive.membershipChange)", + "membership": "\(state.membership)", + ]) }) - self.tracelog(context, .send(to: offer.originNode.node), message: accept) - promise.succeed(.accept(accept)) // This association may mean that we've "replaced" a previously known node of the same host:port, // In case of such replacement we must down and terminate the association of the previous node. + // + // This MUST be called before we complete the new association as it may need to terminate the old one. self.handlePotentialAssociatedMemberReplacement(directive: directive, accept: accept, context: context, state: &state) + // Only now we can succeed the accept promise (as the old one has been terminated and cleared) + self.tracelog(context, .send(to: offer.originNode.node), message: accept) + promise.succeed(.accept(accept)) + // publish any cluster events this association caused. // As the new association is stored, any reactions to these events will use the right underlying connection state.events.publish(.membershipChange(directive.membershipChange)) // TODO: need a test where a leader observes a replacement, and we ensure that it does not end up signalling up or removal twice? @@ -856,11 +896,15 @@ extension ClusterShell { // 1.1) This association may mean that we've "replaced" a previously known node of the same host:port, // In case of such replacement we must down and terminate the association of the previous node. + // // This MUST be called before we complete the new association as it may need to terminate the old one. self.handlePotentialAssociatedMemberReplacement(directive: directive, accept: inboundAccept, context: context, state: &state) // 2) Store the (now completed) association first, as it may be immediately used by remote ActorRefs attempting to send to the remoteNode self.completeAssociation(directive) - state.log.debug("Associated with: \(reflecting: handshakeCompleted.remoteNode); Membership change: \(directive.membershipChange), resulting in: \(state.membership)") + state.log.debug("Associated with: \(reflecting: handshakeCompleted.remoteNode)", metadata: [ + "membership/change": "\(directive.membershipChange)", + "membership": "\(state.membership)", + ]) // 3) publish any cluster events this association caused. // As the new association is stored, any reactions to these events will use the right underlying connection @@ -874,11 +918,6 @@ extension ClusterShell { self.recordMetrics(context.system.metrics, membership: state.membership) - // 5) Finally, signal the handshake future that we've accepted, and become with ready state - let accept: Wire.HandshakeAccept = handshakeCompleted.makeAccept(whenHandshakeReplySent: { () in - self.completeAssociation(directive) - }) - handshakeCompleted.whenCompleted?.succeed(.accept(accept)) return self.ready(state: state) } @@ -894,7 +933,7 @@ extension ClusterShell { // the change was a replacement and thus we need to down the old member (same host:port as the new one), // and terminate its association. - state.log.info("Accepted handshake from [\(accept.targetNode)] which replaces the previously known: \(reflecting: replacedMember).") + state.log.info("Accepted handshake from [\(reflecting: directive.handshake.remoteNode)] which replaces the previously known: [\(reflecting: replacedMember)].") // We MUST be careful to first terminate the association and then store the new one in 2) self.terminateAssociation(context.system, state: &state, replacedMember.node) diff --git a/Sources/DistributedActors/Cluster/ClusterShellState.swift b/Sources/DistributedActors/Cluster/ClusterShellState.swift index 8e525de71..c1c94bf95 100644 --- a/Sources/DistributedActors/Cluster/ClusterShellState.swift +++ b/Sources/DistributedActors/Cluster/ClusterShellState.swift @@ -422,7 +422,7 @@ extension ClusterShellState { // TODO: actions may want to be acted upon, they're like directives, we currently have no such need though; // such actions be e.g. "kill association right away" or "asap tell that node .down" directly without waiting for gossip etc - self.log.trace("Membership updated \(self.membership.prettyDescription(label: "\(self.localNode)")),\n by \(event)") + self.log.trace("Membership updated on [\(self.localNode)] by \(event): \(pretty: self.membership)") return .init(applied: changeWasApplied, leaderActions: leaderActions) } diff --git a/Sources/DistributedActors/Cluster/Downing/DowningStrategy.swift b/Sources/DistributedActors/Cluster/Downing/DowningStrategy.swift index 95e74229f..dca728afe 100644 --- a/Sources/DistributedActors/Cluster/Downing/DowningStrategy.swift +++ b/Sources/DistributedActors/Cluster/Downing/DowningStrategy.swift @@ -106,7 +106,7 @@ internal struct DowningStrategyShell { "Decision to [.down] member [\(member)]!", metadata: self.metadata.merging( [ - "downing/member": "\(member)", + "downing/node": "\(reflecting: member.node)", ], uniquingKeysWith: { l, _ in l } ) diff --git a/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift b/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift index ba829e3ab..e3c549052 100644 --- a/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift +++ b/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift @@ -192,11 +192,6 @@ internal struct HandshakeStateMachine { // self.negotiateCapabilities(...) // TODO: We may want to negotiate other options let completed = CompletedState(fromReceived: self, remoteNode: offer.originNode) -// self.whenCompleted?.succeed(.accept(completed.makeAccept { () in -// self. -// })) -// return completed - return .acceptAndAssociate(completed) } diff --git a/Sources/DistributedActors/Cluster/SWIM/SWIMSettings.swift b/Sources/DistributedActors/Cluster/SWIM/SWIMSettings.swift index 06bbb9979..dc2f93fe7 100644 --- a/Sources/DistributedActors/Cluster/SWIM/SWIMSettings.swift +++ b/Sources/DistributedActors/Cluster/SWIM/SWIMSettings.swift @@ -49,7 +49,7 @@ public struct SWIMSettings { } /// Interval at which gossip messages should be issued. - /// This property sets only a base value of probe interval, which will later be multiplied by `localHealthMultiplier`. + /// This property sets only a base value of probe interval, which will later be multiplied by `SWIMInstance.localHealthMultiplier`. /// - SeeAlso: `maxLocalHealthMultiplier` /// Every `interval` a `fanout` number of gossip messages will be sent. // TODO which fanout? public var probeInterval: TimeAmount = .seconds(1) diff --git a/Sources/DistributedActors/Cluster/Transport/TransportPipelines.swift b/Sources/DistributedActors/Cluster/Transport/TransportPipelines.swift index a4bb7cb5a..9d5693851 100644 --- a/Sources/DistributedActors/Cluster/Transport/TransportPipelines.swift +++ b/Sources/DistributedActors/Cluster/Transport/TransportPipelines.swift @@ -59,7 +59,7 @@ private final class InitiatingHandshakeHandler: ChannelInboundHandler, Removable do { let bytes: ByteBuffer = try proto.serializedByteBuffer(allocator: context.channel.allocator) - // TODO: should we use the serialization infra ourselves here? I guess so... + // TODO: https://github.com/apple/swift-distributed-actors/issues/605 use the serialization infra from the system rather // FIXME: make the promise dance here context.writeAndFlush(self.wrapOutboundOut(bytes), promise: nil) @@ -80,14 +80,14 @@ private final class InitiatingHandshakeHandler: ChannelInboundHandler, Removable let response = try self.readHandshakeResponse(bytes: &bytes) switch response { case .accept(let accept): - self.log.debug("Received handshake accept from: [\(accept.targetNode)]", metadata: metadata) + self.log.debug("Received handshake accept from: [\(reflecting: accept.targetNode)]", metadata: metadata) self.cluster.tell(.inbound(.handshakeAccepted(accept, channel: context.channel))) // handshake is completed, so we remove the handler from the pipeline context.pipeline.removeHandler(self, promise: nil) case .reject(let reject): - self.log.debug("Received handshake reject from: [\(reject.targetNode)] reason: [\(reject.reason)], closing channel.", metadata: metadata) + self.log.debug("Received handshake reject from: [\(reflecting: reject.targetNode)] reason: [\(reject.reason)], closing channel.", metadata: metadata) self.cluster.tell(.inbound(.handshakeRejected(reject))) context.close(promise: nil) } @@ -127,9 +127,11 @@ final class ReceivingHandshakeHandler: ChannelInboundHandler, RemovableChannelHa var bytes = self.unwrapInboundIn(data) let offer = try self.readHandshakeOffer(bytes: &bytes) - self.log.debug("Received handshake offer from: [\(offer.originNode)] with protocol version: [\(offer.version)]", metadata: [ + let metadata: Logger.Metadata = [ "handshake/channel": "\(context.channel)", - ]) + ] + + self.log.debug("Received handshake offer from: [\(reflecting: offer.originNode)] with protocol version: [\(offer.version)]", metadata: metadata) let promise = context.eventLoop.makePromise(of: Wire.HandshakeResponse.self) self.cluster.tell(.inbound(.handshakeOffer(offer, channel: context.channel, handshakeReplyTo: promise))) @@ -138,7 +140,7 @@ final class ReceivingHandshakeHandler: ChannelInboundHandler, RemovableChannelHa switch res { case .success(.accept(let accept)): do { - self.log.debug("Accepting handshake offer from: [\(offer.originNode)]") + self.log.debug("Write accept handshake to: [\(offer.originNode)]", metadata: metadata) try self.writeHandshakeAccept(context, accept) } catch { self.log.error("Failed when sending Handshake.Accept: \(accept), error: \(error)") @@ -147,10 +149,10 @@ final class ReceivingHandshakeHandler: ChannelInboundHandler, RemovableChannelHa case .success(.reject(let reject)): do { - self.log.debug("Rejecting handshake offer from: [\(offer.originNode)] reason: [\(reject.reason)]") + self.log.debug("Write reject handshake offer to: [\(offer.originNode)] reason: [\(reject.reason)]", metadata: metadata) try self.writeHandshakeReject(context, reject) } catch { - self.log.error("Failed when writing \(reject), error: \(error)") + self.log.error("Failed when writing \(reject), error: \(error)", metadata: metadata) context.fireErrorCaught(error) } diff --git a/Sources/DistributedActors/Protobuf/WireProtocol+Serialization.swift b/Sources/DistributedActors/Protobuf/WireProtocol+Serialization.swift index afa9658b2..5e0e1acd2 100644 --- a/Sources/DistributedActors/Protobuf/WireProtocol+Serialization.swift +++ b/Sources/DistributedActors/Protobuf/WireProtocol+Serialization.swift @@ -24,7 +24,7 @@ enum WireEnvelopeError: Error { case emptyRecipient } -// TODO: Implement these using ProtobufRepresentable +// TODO: https://github.com/apple/swift-distributed-actors/issues/605 Implement these using ProtobufRepresentable extension Wire.Envelope { init(_ proto: ProtoEnvelope) throws { diff --git a/Sources/DistributedActors/String+Extensions.swift b/Sources/DistributedActors/String+Extensions.swift index 17042037d..89b82fec5 100644 --- a/Sources/DistributedActors/String+Extensions.swift +++ b/Sources/DistributedActors/String+Extensions.swift @@ -46,12 +46,16 @@ internal extension String.StringInterpolation { // MARK: String Interpolation: reflecting: internal extension String.StringInterpolation { - mutating func appendInterpolation(reflecting subject: CustomDebugStringConvertible) { - self.appendLiteral("[\(String(reflecting: subject))]") + mutating func appendInterpolation(pretty subject: CustomPrettyStringConvertible) { + self.appendLiteral(subject.prettyDescription) } - mutating func appendInterpolation(reflecting subject: Any.Type) { - self.appendLiteral("[\(String(reflecting: subject))]") + mutating func appendInterpolation(reflecting subject: Any?) { + self.appendLiteral(String(reflecting: subject)) + } + + mutating func appendInterpolation(reflecting subject: Any) { + self.appendLiteral(String(reflecting: subject)) } } diff --git a/Sources/DistributedActorsTestKit/Cluster/ClusteredNodesTestBase.swift b/Sources/DistributedActorsTestKit/Cluster/ClusteredNodesTestBase.swift index ee053ad67..fc38cdb7a 100644 --- a/Sources/DistributedActorsTestKit/Cluster/ClusteredNodesTestBase.swift +++ b/Sources/DistributedActorsTestKit/Cluster/ClusteredNodesTestBase.swift @@ -167,7 +167,7 @@ extension ClusteredNodesTestBase { system.cluster.ref.tell(.query(.currentMembership(p.ref))) let membership = try! p.expectMessage() - let info = membership.prettyDescription(label: String(reflecting: system.cluster.node)) + let info = "Membership on [\(reflecting: system.cluster.node)]: \(membership.prettyDescription)" p.stop() diff --git a/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift b/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift index 556aa81b5..71cf24ac3 100644 --- a/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift +++ b/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift @@ -18,6 +18,13 @@ import NIO import XCTest final class ClusterAssociationTests: ClusteredNodesTestBase { + override func configureLogCapture(settings: inout LogCapture.Settings) { + settings.excludeActorPaths = [ + "/system/replicator", + "/system/cluster/swim", + ] + } + // ==== ------------------------------------------------------------------------------------------------------------ // MARK: Happy path, accept association @@ -87,8 +94,7 @@ final class ClusterAssociationTests: ClusteredNodesTestBase { // ==== ------------------------------------------------------------------------------------------------------------ // MARK: Joining into existing cluster - // FIXME: unlock this test // revisit - func fixme_association_sameAddressNodeJoin_shouldOverrideExistingNode() throws { + func test_association_sameAddressNodeJoin_shouldOverrideExistingNode() throws { try shouldNotThrow { let (first, second) = self.setUpPair() @@ -127,31 +133,32 @@ final class ClusterAssociationTests: ClusteredNodesTestBase { } func test_association_shouldAllowSendingToRemoteReference() throws { - let (local, remote) = self.setUpPair() - - let probeOnRemote = self.testKit(remote).spawnTestProbe(expecting: String.self) - let refOnRemoteSystem: ActorRef = try remote.spawn( - "remoteAcquaintance", - .receiveMessage { message in - probeOnRemote.tell("forwarded:\(message)") - return .same - } - ) - - local.cluster.join(node: remote.cluster.node.node) - - try assertAssociated(local, withExactly: remote.settings.cluster.uniqueBindNode) - - // DO NOT TRY THIS AT HOME; we do this since we have no receptionist which could offer us references - // first we manually construct the "right remote path", DO NOT ABUSE THIS IN REAL CODE (please) :-) - let uniqueRemoteAddress = ActorAddress(node: remote.cluster.node, path: refOnRemoteSystem.path, incarnation: refOnRemoteSystem.address.incarnation) - // to then obtain a remote ref ON the `system`, meaning that the node within uniqueRemoteAddress is a remote one - let resolvedRef = self.resolveRef(local, type: String.self, address: uniqueRemoteAddress, on: remote) - // the resolved ref is a local resource on the `system` and points via the right association to the remote actor - // inside system `remote`. Sending messages to a ref constructed like this will make the messages go over remoting. - resolvedRef.tell("HELLO") - - try probeOnRemote.expectMessage("forwarded:HELLO") + try shouldNotThrow { + let (local, remote) = self.setUpPair() + + let probeOnRemote = self.testKit(remote).spawnTestProbe(expecting: String.self) + let refOnRemoteSystem: ActorRef = try remote.spawn( + "remoteAcquaintance", + .receiveMessage { message in + probeOnRemote.tell("forwarded:\(message)") + return .same + } + ) + + local.cluster.join(node: remote.cluster.node.node) + + try assertAssociated(local, withExactly: remote.settings.cluster.uniqueBindNode) + + // first we manually construct the "right remote path"; Don't do this in normal production code + let uniqueRemoteAddress = ActorAddress(node: remote.cluster.node, path: refOnRemoteSystem.path, incarnation: refOnRemoteSystem.address.incarnation) + // to then obtain a remote ref ON the `system`, meaning that the node within uniqueRemoteAddress is a remote one + let resolvedRef = self.resolveRef(local, type: String.self, address: uniqueRemoteAddress, on: remote) + // the resolved ref is a local resource on the `system` and points via the right association to the remote actor + // inside system `remote`. Sending messages to a ref constructed like this will make the messages go over remoting. + resolvedRef.tell("HELLO") + + try probeOnRemote.expectMessage("forwarded:HELLO") + } } // ==== ------------------------------------------------------------------------------------------------------------ @@ -168,8 +175,8 @@ final class ClusterAssociationTests: ClusteredNodesTestBase { first.cluster.ref.tell(.command(.handshakeWith(second.cluster.node.node, replyTo: firstProbe.ref))) second.cluster.ref.tell(.command(.handshakeWith(first.cluster.node.node, replyTo: secondProbe.ref))) -// _ = try firstProbe.expectMessage() -// _ = try secondProbe.expectMessage() + _ = try firstProbe.expectMessage() + _ = try secondProbe.expectMessage() try assertAssociated(first, withExactly: second.settings.cluster.uniqueBindNode) try assertAssociated(second, withExactly: first.settings.cluster.uniqueBindNode) diff --git a/Tests/DistributedActorsTests/MembershipTests.swift b/Tests/DistributedActorsTests/MembershipTests.swift index 2f305e8f7..7bb4c3b83 100644 --- a/Tests/DistributedActorsTests/MembershipTests.swift +++ b/Tests/DistributedActorsTests/MembershipTests.swift @@ -364,6 +364,21 @@ final class MembershipTests: XCTestCase { } } + func test_mark_status_whenReplacingWithNewNode() { + let one = Cluster.Member(node: UniqueNode(node: Node(systemName: "System", host: "1.1.1.1", port: 1001), nid: .random()), status: .joining) + var two = Cluster.Member(node: UniqueNode(node: Node(systemName: "System", host: "2.2.2.2", port: 2222), nid: .random()), status: .up) + let twoReplacement = Cluster.Member(node: UniqueNode(node: Node(systemName: "System", host: "2.2.2.2", port: 2222), nid: .random()), status: .joining) + + var membership: Cluster.Membership = [one, two] + + let changed = membership.mark(twoReplacement.node, as: .joining)! + changed.member.node.shouldEqual(twoReplacement.node) + changed.toStatus.isJoining.shouldBeTrue() + + two.status = .down + membership.shouldEqual([one, two, twoReplacement]) // `twoReplacement` replacement remains joining; is unchanged by mark performed to `two` + } + func test_replacement_changeCreation() { var existing = self.memberA existing.status = .joining From a26a8a68bf3bd2deda8f28af529d73d0425ff516 Mon Sep 17 00:00:00 2001 From: Konrad `ktoso` Malawski Date: Fri, 1 May 2020 16:39:08 +0900 Subject: [PATCH 15/18] INTENSE simplification, drop the whenCompleted future -- it's causing much confusion and in one of many code paths we get it slightly wrong -- and it's really only used in tests... not entirely worth it -- we will revive when we need it --- .../Cluster/Association.swift | 3 - .../Cluster/ClusterControl.swift | 41 +++++- .../Cluster/ClusterShell.swift | 122 ++++++------------ .../Cluster/ClusterShellState.swift | 40 +++--- .../Cluster/HandshakeStateMachine.swift | 56 ++++---- .../Cluster/AssociationClusteredTests.swift | 88 ++++--------- .../RemotingHandshakeStateMachineTests.swift | 14 +- 7 files changed, 149 insertions(+), 215 deletions(-) diff --git a/Sources/DistributedActors/Cluster/Association.swift b/Sources/DistributedActors/Cluster/Association.swift index 96d919b36..878c57f79 100644 --- a/Sources/DistributedActors/Cluster/Association.swift +++ b/Sources/DistributedActors/Cluster/Association.swift @@ -179,13 +179,10 @@ extension Association { self.lock.withLockVoid { switch self.state { case .associating(let sendQueue): - pprint("SEND ENQUEUE = \(envelope)") sendQueue.enqueue(envelope) case .associated(let channel): - pprint("SEND NOW = \(envelope)") channel.writeAndFlush(envelope, promise: promise) case .tombstone(let deadLetters): - pprint("SEND DEAD = \(envelope)") deadLetters.tell(.init(envelope.underlyingMessage, recipient: envelope.recipient)) } } diff --git a/Sources/DistributedActors/Cluster/ClusterControl.swift b/Sources/DistributedActors/Cluster/ClusterControl.swift index d159d8964..549792c8d 100644 --- a/Sources/DistributedActors/Cluster/ClusterControl.swift +++ b/Sources/DistributedActors/Cluster/ClusterControl.swift @@ -70,15 +70,23 @@ public struct ClusterControl { self.settings.uniqueBindNode } + /// Instructs the cluster to join the actor system located listening on the passed in host-port pair. + /// + /// There is no specific need to "wait until joined" before one can attempt to send to references located on the cluster member, + /// as message sends will be buffered until the node associates and joins. public func join(host: String, port: Int) { self.join(node: Node(systemName: "sact", host: host, port: port)) } + /// Instructs the cluster to join the actor system located listening on the passed in host-port pair. + /// + /// There is no specific need to "wait until joined" before one can attempt to send to references located on the cluster member, + /// as message sends will be buffered until the node associates and joins. public func join(node: Node) { - self.ref.tell(.command(.initJoin(node))) + self.ref.tell(.command(.handshakeWith(node))) } - /// Usually NOT to be used, as having an instance of a `UniqueNode` in hand + /// Usually not to be used, as having an instance of a `UniqueNode` in hand /// is normally only possible after a handshake with the remote node has completed. /// /// However, in local testing scenarios, where the two nodes are executing in the same process (e.g. in a test), @@ -89,15 +97,42 @@ public struct ClusterControl { self.join(node: node.node) } + /// Gracefully + /// + /// TODO: no graceful steps implemented today yet) leave the cluster. + /// TODO: leave should perhaps return a future or something to await on. public func leave() { self.ref.tell(.command(.downCommand(self.node.node))) } - /// Mark as `Cluster.MemberStatus.down` _any_ incarnation of a member matching the passed in `node`. + /// Mark *any* currently known member as `Cluster.MemberStatus.down`. + /// + /// Beware that this API is not very precise and, if possible, the `down(Cluster.Member)` is preferred, as it indicates + /// the downing intent of a *specific* actor system instance, rather than any system running on the given host-port pair. + /// + /// This action can be performed by any member of the cluster and is immediately effective locally, as well as spread + /// to other cluster members which will accept is as truth (even if they cal still reach the member and consider it as `.up` etc). + /// + /// Note that once all members have seen the downed node as `.down` it will be completely *removed* from the membership + /// and a tombstone will be stored to prevent it from ever "re-joining" the same cluster. New instances on the same host-port + /// pair however are accepted to join the cluster (though technically this is a newly joining node, not really a "re-join"). + /// + /// - SeeAlso: `Cluster.MemberStatus` for more discussion about what the `.down` status implies. + public func down(node: Node) { self.ref.tell(.command(.downCommand(node))) } + /// Mark the passed in `Cluster.Member` as `Cluster.MemberStatus` `.down`. + /// + /// This action can be performed by any member of the cluster and is immediately effective locally, as well as spread + /// to other cluster members which will accept is as truth (even if they cal still reach the member and consider it as `.up` etc). + /// + /// Note that once all members have seen the downed node as `.down` it will be completely *removed* from the membership + /// and a tombstone will be stored to prevent it from ever "re-joining" the same cluster. New instances on the same host-port + /// pair however are accepted to join the cluster (though technically this is a newly joining node, not really a "re-join"). + /// + /// - SeeAlso: `Cluster.MemberStatus` for more discussion about what the `.down` status implies. public func down(member: Cluster.Member) { self.ref.tell(.command(.downCommandMember(member))) } diff --git a/Sources/DistributedActors/Cluster/ClusterShell.swift b/Sources/DistributedActors/Cluster/ClusterShell.swift index 60a96b022..d06098d81 100644 --- a/Sources/DistributedActors/Cluster/ClusterShell.swift +++ b/Sources/DistributedActors/Cluster/ClusterShell.swift @@ -79,7 +79,7 @@ internal class ClusterShell { /// We're trying to send to `node` yet it has no association (not even in progress), /// thus we need to kick it off. Once it completes it will .completeAssociation() on the stored one (here in the field in Shell). - self.ref.tell(.command(.handshakeWith(node.node, replyTo: nil))) + self.ref.tell(.command(.handshakeWith(node.node))) return .association(association) } @@ -96,6 +96,7 @@ internal class ClusterShell { /// To be invoked by cluster shell whenever handshake is accepted, creating a completed association. /// Causes messages to be flushed onto the new associated channel. private func completeAssociation(_ associated: ClusterShellState.AssociatedDirective) { + // 1) Complete and store the association self._associationsLock.withLockVoid { let association = self._associations[associated.handshake.remoteNode] ?? Association(selfNode: associated.handshake.localNode, remoteNode: associated.handshake.remoteNode) @@ -104,6 +105,9 @@ internal class ClusterShell { self._associations[associated.handshake.remoteNode] = association } + + // 2) Ensure the failure detector knows about this node + self._swimRef?.tell(.local(.monitor(associated.handshake.remoteNode))) } /// Performs all cleanups related to terminating an association: @@ -307,13 +311,11 @@ internal class ClusterShell { // this is basically our API internally for this system enum CommandMessage: NonTransportableActorMessage, SilentDeadLetter { - /// Initiate the joining procedure for the given `Node`, this will result in attempting a handshake, - /// as well as notifying the underlying failure detector (e.g. SWIM) about the node once shook hands with it. - case initJoin(Node) - /// Connect and handshake with remote `Node`, obtaining an `UniqueNode` in the process. /// Once the handshake is completed, reply to `replyTo` with the handshake result, and also mark the unique node as `.joining`. - case handshakeWith(Node, replyTo: ActorRef?) + /// + /// If one is present, the underlying failure detector will be asked to monitor this node as well. + case handshakeWith(Node) case retryHandshake(HandshakeStateMachine.InitiatedState) case failureDetectorReachabilityChanged(UniqueNode, Cluster.MemberReachability) @@ -453,11 +455,8 @@ extension ClusterShell { state.tracelog(.inbound, message: command) switch command { - case .initJoin(let node): - return self.onInitJoin(context, state: state, joining: node) - - case .handshakeWith(let node, let replyTo): - return self.beginHandshake(context, state, with: node, replyTo: replyTo) + case .handshakeWith(let node): + return self.beginHandshake(context, state, with: node) case .retryHandshake(let initiated): return self.retryHandshake(context, state, initiated: initiated) @@ -636,12 +635,11 @@ extension ClusterShell { /// Upon successful handshake, the `replyTo` actor shall be notified with its result, as well as the handshaked-with node shall be marked as `.joining`. /// /// Handshakes are currently not performed concurrently but one by one. - internal func beginHandshake(_ context: ActorContext, _ state: ClusterShellState, with remoteNode: Node, replyTo: ActorRef?) -> Behavior { + internal func beginHandshake(_ context: ActorContext, _ state: ClusterShellState, with remoteNode: Node) -> Behavior { var state = state guard remoteNode != state.localNode.node else { state.log.debug("Ignoring attempt to handshake with myself; Could have been issued as confused attempt to handshake as induced by discovery via gossip?") - replyTo?.tell(.failure(.init(node: remoteNode, message: "Would have attempted handshake with self node, aborted handshake."))) return .same } @@ -650,38 +648,27 @@ extension ClusterShell { state.log.debug("Association already allocated for remote: \(reflecting: remoteNode), existing association: [\(existingAssociation)]") switch existingAssociation.state { case .associating: + () // continue, we may be the first beginHandshake (as associations may be ensured outside of actor context) - existingAssociation.enqueueCompletionTask { - replyTo?.tell(.success(state.localNode)) - } +//// existingAssociation.enqueueCompletionTask { +// replyTo?.tell(.success(state.localNode)) +// } case .associated: // return , we've been successful already - replyTo?.tell(.success(existingAssociation.remoteNode)) +// replyTo?.tell(.success(existingAssociation.remoteNode)) return .same case .tombstone: - replyTo?.tell(.failure( - HandshakeConnectionError( - node: existingAssociation.remoteNode.node, - message: "Existing association for \(existingAssociation.remoteNode) is already a tombstone! Must not complete association." - ) - )) +// replyTo?.tell(.failure( +// HandshakeConnectionError( +// node: existingAssociation.remoteNode.node, +// message: "Existing association for \(existingAssociation.remoteNode) is already a tombstone! Must not complete association." +// ) +// )) return .same } } - let whenHandshakeComplete = state.eventLoopGroup.next().makePromise(of: Wire.HandshakeResponse.self) - whenHandshakeComplete.futureResult.whenComplete { result in - switch result { - case .success(.accept(let accept)): - replyTo?.tell(.success(accept.originNode)) - case .success(.reject(let reject)): - replyTo?.tell(.failure(HandshakeConnectionError(node: remoteNode, message: reject.reason))) - case .failure(let error): - replyTo?.tell(HandshakeResult.failure(HandshakeConnectionError(node: remoteNode, message: "\(error)"))) - } - } - - let handshakeState = state.beginHandshake(with: remoteNode, whenCompleted: whenHandshakeComplete) + let handshakeState = state.initHandshake(with: remoteNode) // we MUST register the intention of shaking hands with remoteAddress before obtaining the connection, // in order to let the fsm handle any retry decisions in face of connection failures et al. @@ -852,7 +839,7 @@ extension ClusterShell { ) case .giveUpOnHandshake: if let hsmState = state.closeOutboundHandshakeChannel(with: remoteNode) { - self.notifyHandshakeFailure(state: hsmState, node: remoteNode, error: error) + state.log.warning("Giving up on handshake: \(hsmState)") } } @@ -947,6 +934,8 @@ extension ClusterShell { } private func onHandshakeRejected(_ context: ActorContext, _ state: ClusterShellState, _ reject: Wire.HandshakeReject) -> Behavior { + var state = state + // we MAY be seeing a handshake failure from a 2 nodes concurrently shaking hands on 2 connections, // and we decided to tie-break and kill one of the connections. As such, the handshake COMPLETED successfully but // on the other connection; and the terminated one may yield an error (e.g. truncation error during proto parsing etc), @@ -957,7 +946,6 @@ extension ClusterShell { "Handshake rejected by [\(reject.targetNode)], it was associating and is now tombstoned", metadata: state.metadataForHandshakes(uniqueNode: reject.targetNode, error: nil) ) - var state = state self.terminateAssociation(context.system, state: &state, reject.targetNode) return self.ready(state: state) } @@ -976,16 +964,7 @@ extension ClusterShell { metadata: state.metadataForHandshakes(uniqueNode: reject.targetNode, error: nil) ) - // TODO: back off and retry, give up after some attempts; count which failure this was, update the handshake state - - if let handshakeState = state.handshakeInProgress(with: reject.targetNode.node) { - self.notifyHandshakeFailure( - state: handshakeState, - node: reject.targetNode.node, - error: HandshakeError.targetRejectedHandshake(selfNode: state.localNode, remoteNode: reject.targetNode, message: reject.reason) - ) - } // else, seems that node successfully associated and this may be for a previous connection - + // FIXME: don't retry on rejections; those are final; just failures are not, clarify this return .same } @@ -1055,18 +1034,18 @@ extension ClusterShell { return self.ready(state: self.onDownCommand(context, state: state, member: myselfMember)) } - private func notifyHandshakeFailure(state: HandshakeStateMachine.State, node: Node, error: Error) { - switch state { - case .initiated(let initiated): - initiated.whenCompleted?.fail(HandshakeConnectionError(node: node, message: "\(error)")) - case .wasOfferedHandshake(let offered): - offered.whenCompleted?.fail(HandshakeConnectionError(node: node, message: "\(error)")) - case .completed(let completed): - completed.whenCompleted?.fail(HandshakeConnectionError(node: node, message: "\(error)")) - case .inFlight: - preconditionFailure("An in-flight marker state should never be stored, yet was encountered in \(#function)") - } - } +// private func notifyHandshakeFailure(state: HandshakeStateMachine.State, node: Node, error: Error) { +// switch state { +// case .initiated(let initiated): +// initiated.whenCompleted.fail(HandshakeConnectionError(node: node, message: "\(error)")) +// case .wasOfferedHandshake(let offered): +// offered.whenCompleted.fail(HandshakeConnectionError(node: node, message: "\(error)")) +// case .completed(let completed): +// completed.whenCompleted.fail(HandshakeConnectionError(node: node, message: "\(error)")) +// case .inFlight: +// preconditionFailure("An in-flight marker state should never be stored, yet was encountered in \(#function)") +// } +// } } // ==== ---------------------------------------------------------------------------------------------------------------- @@ -1097,29 +1076,6 @@ extension ClusterShell { // MARK: Handling cluster membership changes extension ClusterShell { - /// Ensure an association, and let SWIM know about it - func onInitJoin(_ context: ActorContext, state _: ClusterShellState, joining node: Node) -> Behavior { - let handshakeResultAnswer: AskResponse = context.myself.ask(for: HandshakeResult.self, timeout: .seconds(3)) { - Message.command(.handshakeWith(node, replyTo: $0)) - } - - context.onResultAsync(of: handshakeResultAnswer, timeout: .effectivelyInfinite) { (res: Result) in - switch res { - case .success(.success(let uniqueNode)): - context.log.debug("Associated \(uniqueNode), informing SWIM to monitor this node.") - self._swimRef?.tell(.local(.monitor(uniqueNode))) - return .same // .same, since state was modified since inside the handshakeWith (!) - case .success(.failure(let error)): - context.log.debug("Handshake with \(reflecting: node) failed: \(error)") - return .same - case .failure(let error): - context.log.debug("Handshake with \(reflecting: node) failed: \(error)") - return .same - } - } - - return .same - } func onReachabilityChange( _ context: ActorContext, diff --git a/Sources/DistributedActors/Cluster/ClusterShellState.swift b/Sources/DistributedActors/Cluster/ClusterShellState.swift index c1c94bf95..08061928a 100644 --- a/Sources/DistributedActors/Cluster/ClusterShellState.swift +++ b/Sources/DistributedActors/Cluster/ClusterShellState.swift @@ -120,27 +120,24 @@ extension ClusterShellState { /// /// This MAY return `inFlight`, in which case it means someone already initiated a handshake with given node, /// and we should _do nothing_ and trust that our `whenCompleted` will be notified when the already in-flight handshake completes. - mutating func beginHandshake(with remoteNode: Node, whenCompleted: EventLoopPromise) -> HandshakeStateMachine.State { + mutating func initHandshake(with remoteNode: Node) -> HandshakeStateMachine.State { if let handshakeState = self.handshakeInProgress(with: remoteNode) { switch handshakeState { - case .initiated(let state): - state.whenCompleted?.futureResult.cascade(to: whenCompleted) - case .completed(let state): - state.whenCompleted?.futureResult.cascade(to: whenCompleted) - case .wasOfferedHandshake(let state): - state.whenCompleted?.futureResult.cascade(to: whenCompleted) + case .initiated: + return .inFlight(HandshakeStateMachine.InFlightState(state: self)) + case .completed: + return .inFlight(HandshakeStateMachine.InFlightState(state: self)) + case .wasOfferedHandshake: + return .inFlight(HandshakeStateMachine.InFlightState(state: self)) case .inFlight: fatalError("An inFlight may never be stored, yet seemingly was! Offending state: \(self) for node \(remoteNode)") } - - return .inFlight(HandshakeStateMachine.InFlightState(state: self, whenCompleted: whenCompleted)) } let initiated = HandshakeStateMachine.InitiatedState( settings: self.settings, localNode: self.localNode, - connectTo: remoteNode, - whenCompleted: whenCompleted + connectTo: remoteNode ) let handshakeState = HandshakeStateMachine.State.initiated(initiated) self._handshakes[remoteNode] = handshakeState @@ -153,10 +150,11 @@ extension ClusterShellState { if case .some(.initiated(let existingInitiated)) = handshakeInProgress { if existingInitiated.remoteNode != initiated.remoteNode { - fatalError(""" - onHandshakeChannelConnected MUST be called with the existing ongoing initiated - handshake! Existing: \(existingInitiated), passed in: \(initiated). - """) + fatalError( + """ + onHandshakeChannelConnected MUST be called with the existing ongoing initiated \ + handshake! Existing: \(existingInitiated), passed in: \(initiated). + """) } if existingInitiated.channel != nil { fatalError("onHandshakeChannelConnected should only be invoked once on an initiated state; yet seems the state already has a channel! Was: \(String(reflecting: handshakeInProgress))") @@ -232,15 +230,15 @@ extension ClusterShellState { /// Winner: Keeps the outgoing connection, negotiates and replies accept/reject on the "incoming" connection from the remote node. /// Loser: Drops the incoming connection and waits for Winner's decision. mutating func onIncomingHandshakeOffer(offer: Wire.HandshakeOffer, incomingChannel: Channel) -> OnIncomingHandshakeOfferDirective { - func prepareNegotiation(promise: EventLoopPromise? = nil) -> OnIncomingHandshakeOfferDirective { - let fsm = HandshakeStateMachine.HandshakeReceivedState(state: self, offer: offer, whenCompleted: promise) + func prepareNegotiation0() -> OnIncomingHandshakeOfferDirective { + let fsm = HandshakeStateMachine.HandshakeOfferReceivedState(state: self, offer: offer) self._handshakes[offer.originNode.node] = .wasOfferedHandshake(fsm) return .negotiateIncoming(fsm) } guard let inProgress = self._handshakes[offer.originNode.node] else { // no other concurrent handshakes in progress; good, this is happy path, so we simply continue our negotiation - return prepareNegotiation() + return prepareNegotiation0() } switch inProgress { @@ -273,7 +271,7 @@ extension ClusterShellState { ) } - return prepareNegotiation(promise: initiated.whenCompleted) + return prepareNegotiation0() } else { // we "lost", the other node will send the accept; when it does, the will complete the future. @@ -284,7 +282,7 @@ extension ClusterShellState { // Situations: // - it could be that the remote re-sent their offer before it received our accept? // - maybe remote did not receive our accept/reject and is trying again? - return prepareNegotiation() + return prepareNegotiation0() // --- these are never stored ---- case .inFlight(let inFlight): @@ -295,7 +293,7 @@ extension ClusterShellState { } enum OnIncomingHandshakeOfferDirective { - case negotiateIncoming(HandshakeStateMachine.HandshakeReceivedState) + case negotiateIncoming(HandshakeStateMachine.HandshakeOfferReceivedState) /// An existing handshake with given peer is already in progress, /// do not negotiate but rest assured that the association will be handled properly by the already ongoing process. case abortIncomingDueToConcurrentHandshake diff --git a/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift b/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift index e3c549052..0d70bbf1f 100644 --- a/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift +++ b/Sources/DistributedActors/Cluster/HandshakeStateMachine.swift @@ -70,7 +70,6 @@ internal struct HandshakeStateMachine { let remoteNode: Node let localNode: UniqueNode - let whenCompleted: EventLoopPromise? /// Channel which was established when we initiated the handshake (outgoing connection), /// which may want to be closed when we `abortHandshake` and want to kill the related outgoing connection as we do so. @@ -81,15 +80,13 @@ internal struct HandshakeStateMachine { // TODO: counter for how many times to retry associating (timeouts) init( - settings: ClusterSettings, localNode: UniqueNode, connectTo remoteNode: Node, - whenCompleted: EventLoopPromise? + settings: ClusterSettings, localNode: UniqueNode, connectTo remoteNode: Node ) { precondition(localNode.node != remoteNode, "MUST NOT attempt connecting to own bind address. Address: \(remoteNode)") self.settings = settings self.backoff = settings.associationHandshakeBackoff self.localNode = localNode self.remoteNode = remoteNode - self.whenCompleted = whenCompleted } func makeOffer() -> Wire.HandshakeOffer { @@ -124,7 +121,6 @@ internal struct HandshakeStateMachine { remoteNode: \(self.remoteNode), \ localNode: \(self.localNode), \ backoff: \(self.backoff), \ - whenCompleted: \(optional: self.whenCompleted), \ channel: \(optional: self.channel)\ ) """ @@ -137,11 +133,8 @@ internal struct HandshakeStateMachine { internal struct InFlightState { private let state: ReadOnlyClusterState - let whenCompleted: EventLoopPromise - - init(state: ReadOnlyClusterState, whenCompleted: EventLoopPromise) { + init(state: ReadOnlyClusterState) { self.state = state - self.whenCompleted = whenCompleted } } @@ -149,7 +142,7 @@ internal struct HandshakeStateMachine { // MARK: Handshake Received /// Initial state for server side of handshake. - internal struct HandshakeReceivedState { + internal struct HandshakeOfferReceivedState { private let state: ReadOnlyClusterState let offer: Wire.HandshakeOffer @@ -161,12 +154,9 @@ internal struct HandshakeStateMachine { self.state.settings.protocolVersion } - let whenCompleted: EventLoopPromise? - - init(state: ReadOnlyClusterState, offer: Wire.HandshakeOffer, whenCompleted: EventLoopPromise?) { + init(state: ReadOnlyClusterState, offer: Wire.HandshakeOffer) { self.state = state self.offer = offer - self.whenCompleted = whenCompleted } func negotiate() -> HandshakeStateMachine.NegotiateDirective { @@ -174,17 +164,17 @@ internal struct HandshakeStateMachine { let error = HandshakeError.targetHandshakeAddressMismatch(self.offer, selfNode: self.boundAddress) let rejectedState = RejectedState(fromReceived: self, remoteNode: self.offer.originNode, error: error) - self.whenCompleted?.succeed(.reject(rejectedState.makeReject(whenHandshakeReplySent: { () in - self.state.log.trace("Done rejecting handshake.") // TODO: something more, terminate the association? - }))) +// self.whenCompleted.succeed(.reject(rejectedState.makeReject(whenHandshakeReplySent: { () in +// self.state.log.trace("Done rejecting handshake.") // TODO: something more, terminate the association? +// }))) return .rejectHandshake(rejectedState) } // negotiate version if let rejectedState = self.negotiateVersion(local: self.protocolVersion, remote: self.offer.version) { - self.whenCompleted?.succeed(.reject(rejectedState.makeReject(whenHandshakeReplySent: { () in - self.state.log.trace("Done rejecting handshake.") // TODO: something more, terminate the association? - }))) +// self.whenCompleted.succeed(.reject(rejectedState.makeReject(whenHandshakeReplySent: { () in +// self.state.log.trace("Done rejecting handshake.") // TODO: something more, terminate the association? +// }))) return .rejectHandshake(rejectedState) } @@ -216,28 +206,28 @@ internal struct HandshakeStateMachine { let protocolVersion: Version var remoteNode: UniqueNode var localNode: UniqueNode - let whenCompleted: EventLoopPromise? +// let whenCompleted: EventLoopPromise // let unique association ID? // State Transition used by Client Side of initial Handshake. // // Since the client is the one who initiates the handshake, once it receives an Accept containing the remote unique node // it may immediately transition to the completed state. - init(fromInitiated state: InitiatedState, remoteNode: UniqueNode) { - precondition(state.localNode != remoteNode, "Node [\(state.localNode)] attempted to create association with itself.") - self.protocolVersion = state.protocolVersion + init(fromInitiated initiated: InitiatedState, remoteNode: UniqueNode) { + precondition(initiated.localNode != remoteNode, "Node [\(initiated.localNode)] attempted to create association with itself.") + self.protocolVersion = initiated.protocolVersion self.remoteNode = remoteNode - self.localNode = state.localNode - self.whenCompleted = state.whenCompleted + self.localNode = initiated.localNode +// self.whenCompleted = initiated.whenCompleted } // State Transition used by Server Side on accepting a received Handshake. - init(fromReceived state: HandshakeReceivedState, remoteNode: UniqueNode) { - precondition(state.boundAddress != remoteNode, "Node [\(state.boundAddress)] attempted to create association with itself.") - self.protocolVersion = state.protocolVersion + init(fromReceived received: HandshakeOfferReceivedState, remoteNode: UniqueNode) { + precondition(received.boundAddress != remoteNode, "Node [\(received.boundAddress)] attempted to create association with itself.") + self.protocolVersion = received.protocolVersion self.remoteNode = remoteNode - self.localNode = state.boundAddress - self.whenCompleted = state.whenCompleted + self.localNode = received.boundAddress +// self.whenCompleted = received.whenCompleted } func makeAccept(whenHandshakeReplySent: @escaping () -> Void) -> Wire.HandshakeAccept { @@ -256,7 +246,7 @@ internal struct HandshakeStateMachine { let remoteNode: UniqueNode let error: HandshakeError - init(fromReceived state: HandshakeReceivedState, remoteNode: UniqueNode, error: HandshakeError) { + init(fromReceived state: HandshakeOfferReceivedState, remoteNode: UniqueNode, error: HandshakeError) { self.protocolVersion = state.protocolVersion self.localNode = state.boundAddress self.remoteNode = remoteNode @@ -280,7 +270,7 @@ internal struct HandshakeStateMachine { /// rather than creating another handshake dance, we will be notified along with the already initiated /// by someone else handshake completes. case inFlight(InFlightState) - case wasOfferedHandshake(HandshakeReceivedState) + case wasOfferedHandshake(HandshakeOfferReceivedState) case completed(CompletedState) } } diff --git a/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift b/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift index 71cf24ac3..d4eeff70f 100644 --- a/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift +++ b/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift @@ -64,31 +64,30 @@ final class ClusterAssociationTests: ClusteredNodesTestBase { } func test_handshake_shouldNotifyOnSuccess() throws { - let (local, remote) = self.setUpPair() - let p = self.testKit(local).spawnTestProbe(expecting: ClusterShell.HandshakeResult.self) - - local.cluster.ref.tell(.command(.handshakeWith(remote.cluster.node.node, replyTo: p.ref))) + try shouldNotThrow { + let (local, remote) = self.setUpPair() - try assertAssociated(local, withExactly: remote.cluster.node) - try assertAssociated(remote, withExactly: local.cluster.node) + local.cluster.ref.tell(.command(.handshakeWith(remote.cluster.node.node))) - try p.expectMessage(.success(remote.cluster.node), within: .seconds(3)) + try assertAssociated(local, withExactly: remote.cluster.node) + try assertAssociated(remote, withExactly: local.cluster.node) + } } func test_handshake_shouldNotifySuccessWhenAlreadyConnected() throws { - let (local, remote) = self.setUpPair() - let p = self.testKit(local).spawnTestProbe(expecting: ClusterShell.HandshakeResult.self) + try shouldNotThrow { + let (local, remote) = self.setUpPair() - local.cluster.ref.tell(.command(.handshakeWith(remote.cluster.node.node, replyTo: p.ref))) + local.cluster.ref.tell(.command(.handshakeWith(remote.cluster.node.node))) - try assertAssociated(local, withExactly: remote.cluster.node) - try assertAssociated(remote, withExactly: local.cluster.node) + try assertAssociated(local, withExactly: remote.cluster.node) + try assertAssociated(remote, withExactly: local.cluster.node) - try p.expectMessage(.success(remote.cluster.node)) + local.cluster.ref.tell(.command(.handshakeWith(remote.cluster.node.node))) - local.cluster.ref.tell(.command(.handshakeWith(remote.cluster.node.node, replyTo: p.ref))) - - try p.expectMessage(.success(remote.cluster.node)) + try assertAssociated(local, withExactly: remote.cluster.node) + try assertAssociated(remote, withExactly: local.cluster.node) + } } // ==== ------------------------------------------------------------------------------------------------------------ @@ -167,16 +166,10 @@ final class ClusterAssociationTests: ClusteredNodesTestBase { func test_association_shouldEstablishSingleAssociationForConcurrentlyInitiatedHandshakes_incoming_outgoing() throws { let (first, second) = self.setUpPair() - let firstProbe = self.testKit(first).spawnTestProbe(expecting: ClusterShell.HandshakeResult.self) - let secondProbe = self.testKit(second).spawnTestProbe(expecting: ClusterShell.HandshakeResult.self) - // here we attempt to make a race where the nodes race to join each other // again, only one association should be created. - first.cluster.ref.tell(.command(.handshakeWith(second.cluster.node.node, replyTo: firstProbe.ref))) - second.cluster.ref.tell(.command(.handshakeWith(first.cluster.node.node, replyTo: secondProbe.ref))) - - _ = try firstProbe.expectMessage() - _ = try secondProbe.expectMessage() + first.cluster.ref.tell(.command(.handshakeWith(second.cluster.node.node))) + second.cluster.ref.tell(.command(.handshakeWith(first.cluster.node.node))) try assertAssociated(first, withExactly: second.settings.cluster.uniqueBindNode) try assertAssociated(second, withExactly: first.settings.cluster.uniqueBindNode) @@ -185,15 +178,9 @@ final class ClusterAssociationTests: ClusteredNodesTestBase { func test_association_shouldEstablishSingleAssociationForConcurrentlyInitiatedHandshakes_outgoing_outgoing() throws { let (first, second) = setUpPair() - let firstProbe = self.testKit(first).spawnTestProbe(expecting: ClusterShell.HandshakeResult.self) - let secondProbe = self.testKit(first).spawnTestProbe(expecting: ClusterShell.HandshakeResult.self) - // we issue two handshakes quickly after each other, both should succeed but there should only be one association established (!) - first.cluster.ref.tell(.command(.handshakeWith(second.cluster.node.node, replyTo: firstProbe.ref))) - first.cluster.ref.tell(.command(.handshakeWith(second.cluster.node.node, replyTo: secondProbe.ref))) - - _ = try firstProbe.expectMessage() - _ = try secondProbe.expectMessage() + first.cluster.ref.tell(.command(.handshakeWith(second.cluster.node.node))) + first.cluster.ref.tell(.command(.handshakeWith(second.cluster.node.node))) try assertAssociated(first, withExactly: second.settings.cluster.uniqueBindNode) try assertAssociated(second, withExactly: first.settings.cluster.uniqueBindNode) @@ -233,25 +220,17 @@ final class ClusterAssociationTests: ClusteredNodesTestBase { } func test_handshake_shouldNotifyOnRejection() throws { - let local = self.setUpNode("local") { - $0.cluster._protocolVersion.major += 1 // handshake will be rejected on major version difference + let local = self.setUpNode("local") { settings in + settings.cluster._protocolVersion.major += 1 // handshake will be rejected on major version difference } let remote = self.setUpNode("remote") - let p = self.testKit(local).spawnTestProbe(expecting: ClusterShell.HandshakeResult.self) - - local.cluster.ref.tell(.command(.handshakeWith(remote.cluster.node.node, replyTo: p.ref))) + local.cluster.ref.tell(.command(.handshakeWith(remote.cluster.node.node))) try assertNotAssociated(system: local, node: remote.cluster.node) try assertNotAssociated(system: remote, node: local.cluster.node) - switch try p.expectMessage() { - case ClusterShell.HandshakeResult.failure(let err): - "\(err)".shouldContain("incompatibleProtocolVersion(local:") - () // ok - default: - throw p.error() - } + try self.capturedLogs(of: local).awaitLogContaining(self.testKit(local), text: "incompatibleProtocolVersion(local:") } // ==== ------------------------------------------------------------------------------------------------------------ @@ -275,27 +254,6 @@ final class ClusterAssociationTests: ClusteredNodesTestBase { local._cluster?._testingOnly_associations.count.shouldEqual(2) } - // FIXME: once initiated, handshake seem to retry until they succeed, that seems - // like a problem and should be fixed. This test should be re-enabled, - // once issue #724 (handshakes should not retry forever) is resolved - func disabled_test_handshake_shouldNotifyOnConnectionFailure() throws { - let local = setUpNode("local") - - let p = self.testKit(local).spawnTestProbe(expecting: ClusterShell.HandshakeResult.self) - - var node = local.cluster.node.node - node.port = node.port + 10 - - local.cluster.ref.tell(.command(.handshakeWith(node, replyTo: p.ref))) // TODO: nicer API - - switch try p.expectMessage(within: .seconds(1)) { - case ClusterShell.HandshakeResult.failure: - () // ok - default: - throw p.error() - } - } - func test_sendingMessageToNotYetAssociatedNode_mustCauseAssociationAttempt() throws { let first = self.setUpNode("first") let second = self.setUpNode("second") diff --git a/Tests/DistributedActorsTests/Cluster/RemotingHandshakeStateMachineTests.swift b/Tests/DistributedActorsTests/Cluster/RemotingHandshakeStateMachineTests.swift index 68242874a..bc483aa98 100644 --- a/Tests/DistributedActorsTests/Cluster/RemotingHandshakeStateMachineTests.swift +++ b/Tests/DistributedActorsTests/Cluster/RemotingHandshakeStateMachineTests.swift @@ -36,11 +36,11 @@ final class RemoteHandshakeStateMachineTests: XCTestCase { } // client - let clientInitiated = HSM.InitiatedState(settings: clientKernel.settings, localNode: clientKernel.localNode, connectTo: serverAddress.node, whenCompleted: nil) + let clientInitiated = HSM.InitiatedState(settings: clientKernel.settings, localNode: clientKernel.localNode, connectTo: serverAddress.node) let offer = clientInitiated.makeOffer() // server - let received = HSM.HandshakeReceivedState(state: serverKernel, offer: offer, whenCompleted: nil) // TODO: test that it completes? + let received = HSM.HandshakeOfferReceivedState(state: serverKernel, offer: offer) // TODO: test that it completes? let serverCompleted: HSM.CompletedState switch received.negotiate() { @@ -74,11 +74,11 @@ final class RemoteHandshakeStateMachineTests: XCTestCase { settings._protocolVersion.patch += 1 } - let clientInitiated = HSM.InitiatedState(settings: clientKernel.settings, localNode: clientKernel.localNode, connectTo: serverAddress.node, whenCompleted: nil) + let clientInitiated = HSM.InitiatedState(settings: clientKernel.settings, localNode: clientKernel.localNode, connectTo: serverAddress.node) let offer = clientInitiated.makeOffer() // server - let received = HSM.HandshakeReceivedState(state: serverKernel, offer: offer, whenCompleted: nil) // TODO: test that it completes? + let received = HSM.HandshakeOfferReceivedState(state: serverKernel, offer: offer) // TODO: test that it completes? // then @@ -99,11 +99,11 @@ final class RemoteHandshakeStateMachineTests: XCTestCase { settings._protocolVersion.major += 1 } - let clientInitiated = HSM.InitiatedState(settings: clientKernel.settings, localNode: clientKernel.localNode, connectTo: serverAddress.node, whenCompleted: nil) + let clientInitiated = HSM.InitiatedState(settings: clientKernel.settings, localNode: clientKernel.localNode, connectTo: serverAddress.node) let offer = clientInitiated.makeOffer() // server - let received = HSM.HandshakeReceivedState(state: serverKernel, offer: offer, whenCompleted: nil) // TODO: test that it completes? + let received = HSM.HandshakeOfferReceivedState(state: serverKernel, offer: offer) // TODO: test that it completes? // then @@ -130,7 +130,7 @@ final class RemoteHandshakeStateMachineTests: XCTestCase { } // client - var clientInitiated = HSM.InitiatedState(settings: clientKernel.settings, localNode: clientKernel.localNode, connectTo: serverAddress.node, whenCompleted: nil) + var clientInitiated = HSM.InitiatedState(settings: clientKernel.settings, localNode: clientKernel.localNode, connectTo: serverAddress.node) guard case .scheduleRetryHandshake = clientInitiated.onHandshakeTimeout() else { throw shouldNotHappen("Expected retry attempt after handshake timeout") From 9e74a11fb5074331b1fa8c8c4f0658d2cd319bf7 Mon Sep 17 00:00:00 2001 From: Konrad `ktoso` Malawski Date: Fri, 1 May 2020 16:42:00 +0900 Subject: [PATCH 16/18] =tests need to use 127.0.0.1 rather than localhost for consistency --- .../Cluster/ClusterShell.swift | 18 ------------------ .../Protobuf/CRDT+SerializationTests.swift | 4 ++-- 2 files changed, 2 insertions(+), 20 deletions(-) diff --git a/Sources/DistributedActors/Cluster/ClusterShell.swift b/Sources/DistributedActors/Cluster/ClusterShell.swift index d06098d81..3b8562816 100644 --- a/Sources/DistributedActors/Cluster/ClusterShell.swift +++ b/Sources/DistributedActors/Cluster/ClusterShell.swift @@ -139,7 +139,6 @@ internal class ClusterShell { traceLog_Remote(system.cluster.node, "Finish terminate association [\(remoteNode)]: Notifying SWIM, .confirmDead") self._swimRef?.tell(.local(.confirmDead(remoteNode))) - let before = state.membership // it is important that we first check the contains; as otherwise we'd re-add a .down member for what was already removed (!) if state.membership.contains(remoteNode) { // Ensure to remove (completely) the member from the Membership, it is not even .leaving anymore. @@ -153,23 +152,6 @@ internal class ClusterShell { } // else: Note that we CANNOT remove() just yet, as we only want to do this when all nodes have seen the down/leaving } -// state.membership.mark(remoteNode, as: .down); remoteNode = sact://second:1026573596@127.0.0.1:9002 -// BEFORE MARK DOWN = LEADER: Member(sact://first@127.0.0.1:9001, status: joining, reachability: reachable) -// sact://first:1954943626@127.0.0.1:9001 STATUS: [joining] -// sact://second-REPLACEMENT:3526768720@127.0.0.1:9002 STATUS: [joining] -// -// AFTER MARK DOWN = LEADER: Member(sact://first@127.0.0.1:9001, status: joining, reachability: reachable) -// sact://first:1954943626@127.0.0.1:9001 STATUS: [joining] -// sact://second-REPLACEMENT:3526768720@127.0.0.1:9002 STATUS: [ down] -// sact://second:1026573596@127.0.0.1:9002 STATUS: [ down] - - - pprint(""" - state.membership.mark(remoteNode, as: .down); remoteNode = \(reflecting: remoteNode) - BEFORE MARK DOWN = \(pretty: before) - AFTER MARK DOWN = \(pretty: state.membership) - """) - // The last thing we attempt to do with the other node is to shoot it, // in case it's a "zombie" that still may receive messages for some reason. ClusterShell.shootTheOtherNodeAndCloseConnection(system: system, targetNodeAssociation: removedAssociation) diff --git a/Tests/DistributedActorsTests/CRDT/Protobuf/CRDT+SerializationTests.swift b/Tests/DistributedActorsTests/CRDT/Protobuf/CRDT+SerializationTests.swift index 206c022e5..2e6fe8f51 100644 --- a/Tests/DistributedActorsTests/CRDT/Protobuf/CRDT+SerializationTests.swift +++ b/Tests/DistributedActorsTests/CRDT/Protobuf/CRDT+SerializationTests.swift @@ -299,7 +299,7 @@ final class CRDTSerializationTests: ActorSystemTestBase { let serialized = try system.serialization.serialize(map) let deserialized = try system.serialization.deserialize(as: CRDT.ORMultiMap.self, from: serialized) - "\(deserialized.replicaID)".shouldContain("actor:sact://CRDTSerializationTests@localhost:9001/user/alpha") + "\(deserialized.replicaID)".shouldContain("actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/alpha") deserialized.state._keys.elements.shouldEqual(["s1", "s2"]) deserialized.state._values.count.shouldEqual(2) @@ -360,7 +360,7 @@ final class CRDTSerializationTests: ActorSystemTestBase { let serialized = try system.serialization.serialize(map) let deserialized = try system.serialization.deserialize(as: CRDT.LWWMap.self, from: serialized) - "\(deserialized.replicaID)".shouldContain("actor:sact://CRDTSerializationTests@localhost:9001/user/alpha") + "\(deserialized.replicaID)".shouldContain("actor:sact://CRDTSerializationTests@127.0.0.1:9001/user/alpha") deserialized.state._keys.elements.shouldEqual(["foo", "bar"]) deserialized.state._values.count.shouldEqual(2) From 423f60b7834863b3529f659e96c0e861103bfd93 Mon Sep 17 00:00:00 2001 From: Konrad `ktoso` Malawski Date: Fri, 1 May 2020 16:46:00 +0900 Subject: [PATCH 17/18] =test cleanups --- .../DistributedActors/Cluster/ClusterControl.swift | 4 ++-- .../DistributedActors/Cluster/ClusterShell.swift | 13 ++++++------- .../Cluster/ClusterShellState.swift | 8 ++++---- .../Cluster/AssociationClusteredTests.swift | 12 ++++++------ Tests/DistributedActorsTests/MembershipTests.swift | 4 ++-- 5 files changed, 20 insertions(+), 21 deletions(-) diff --git a/Sources/DistributedActors/Cluster/ClusterControl.swift b/Sources/DistributedActors/Cluster/ClusterControl.swift index 549792c8d..3774a05c1 100644 --- a/Sources/DistributedActors/Cluster/ClusterControl.swift +++ b/Sources/DistributedActors/Cluster/ClusterControl.swift @@ -99,8 +99,8 @@ public struct ClusterControl { /// Gracefully /// - /// TODO: no graceful steps implemented today yet) leave the cluster. - /// TODO: leave should perhaps return a future or something to await on. + // TODO: no graceful steps implemented today yet) leave the cluster. + // TODO: leave should perhaps return a future or something to await on. public func leave() { self.ref.tell(.command(.downCommand(self.node.node))) } diff --git a/Sources/DistributedActors/Cluster/ClusterShell.swift b/Sources/DistributedActors/Cluster/ClusterShell.swift index 3b8562816..07a9582db 100644 --- a/Sources/DistributedActors/Cluster/ClusterShell.swift +++ b/Sources/DistributedActors/Cluster/ClusterShell.swift @@ -145,9 +145,9 @@ internal class ClusterShell { if state.membership.mark(remoteNode, as: .down) == nil { // it was already removed, nothing to do state.log.trace( - "Terminate association with \(reflecting: remoteNode), yet node not in membership already?", metadata: [ - "cluster/membership": "\(pretty: state.membership)" - ] + "Terminate association with \(reflecting: remoteNode), yet node not in membership already?", metadata: [ + "cluster/membership": "\(pretty: state.membership)", + ] ) } // else: Note that we CANNOT remove() just yet, as we only want to do this when all nodes have seen the down/leaving } @@ -631,8 +631,8 @@ extension ClusterShell { switch existingAssociation.state { case .associating: () - // continue, we may be the first beginHandshake (as associations may be ensured outside of actor context) -//// existingAssociation.enqueueCompletionTask { + // continue, we may be the first beginHandshake (as associations may be ensured outside of actor context) + //// existingAssociation.enqueueCompletionTask { // replyTo?.tell(.success(state.localNode)) // } case .associated: @@ -666,13 +666,13 @@ extension ClusterShell { return self.ready(state: state) } } + internal func retryHandshake(_ context: ActorContext, _ state: ClusterShellState, initiated: HandshakeStateMachine.InitiatedState) -> Behavior { state.log.info("Retry handshake with: \(initiated.remoteNode)") // TODO: update retry counter, perhaps give up return self.connectSendHandshakeOffer(context, state, initiated: initiated) - } func connectSendHandshakeOffer(_ context: ActorContext, _ state: ClusterShellState, initiated: HandshakeStateMachine.InitiatedState) -> Behavior { @@ -1058,7 +1058,6 @@ extension ClusterShell { // MARK: Handling cluster membership changes extension ClusterShell { - func onReachabilityChange( _ context: ActorContext, state: ClusterShellState, diff --git a/Sources/DistributedActors/Cluster/ClusterShellState.swift b/Sources/DistributedActors/Cluster/ClusterShellState.swift index 08061928a..57495ba32 100644 --- a/Sources/DistributedActors/Cluster/ClusterShellState.swift +++ b/Sources/DistributedActors/Cluster/ClusterShellState.swift @@ -151,10 +151,10 @@ extension ClusterShellState { if case .some(.initiated(let existingInitiated)) = handshakeInProgress { if existingInitiated.remoteNode != initiated.remoteNode { fatalError( - """ - onHandshakeChannelConnected MUST be called with the existing ongoing initiated \ - handshake! Existing: \(existingInitiated), passed in: \(initiated). - """) + """ + onHandshakeChannelConnected MUST be called with the existing ongoing initiated \ + handshake! Existing: \(existingInitiated), passed in: \(initiated). + """) } if existingInitiated.channel != nil { fatalError("onHandshakeChannelConnected should only be invoked once on an initiated state; yet seems the state already has a channel! Was: \(String(reflecting: handshakeInProgress))") diff --git a/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift b/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift index d4eeff70f..e4c412ada 100644 --- a/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift +++ b/Tests/DistributedActorsTests/Cluster/AssociationClusteredTests.swift @@ -137,11 +137,11 @@ final class ClusterAssociationTests: ClusteredNodesTestBase { let probeOnRemote = self.testKit(remote).spawnTestProbe(expecting: String.self) let refOnRemoteSystem: ActorRef = try remote.spawn( - "remoteAcquaintance", - .receiveMessage { message in - probeOnRemote.tell("forwarded:\(message)") - return .same - } + "remoteAcquaintance", + .receiveMessage { message in + probeOnRemote.tell("forwarded:\(message)") + return .same + } ) local.cluster.join(node: remote.cluster.node.node) @@ -220,7 +220,7 @@ final class ClusterAssociationTests: ClusteredNodesTestBase { } func test_handshake_shouldNotifyOnRejection() throws { - let local = self.setUpNode("local") { settings in + let local = self.setUpNode("local") { settings in settings.cluster._protocolVersion.major += 1 // handshake will be rejected on major version difference } let remote = self.setUpNode("remote") diff --git a/Tests/DistributedActorsTests/MembershipTests.swift b/Tests/DistributedActorsTests/MembershipTests.swift index 7bb4c3b83..30f5061ec 100644 --- a/Tests/DistributedActorsTests/MembershipTests.swift +++ b/Tests/DistributedActorsTests/MembershipTests.swift @@ -293,7 +293,7 @@ final class MembershipTests: XCTestCase { // as if the fromStatus is not set we may infer it from other places; but in such change, we definitely want it in the `from` change1?.fromStatus.shouldEqual(.joining) change1?.toStatus.shouldEqual(.up) - "\(change1!)".shouldContain("fromStatus: joining, toStatus: up)") + "\(change1!)".shouldContain("1001 :: [joining] -> [ up]") membership.mark(member.node, as: .joining).shouldBeNil() // can't move "back" membership.mark(member.node, as: .up).shouldBeNil() // don't move to "same" @@ -302,7 +302,7 @@ final class MembershipTests: XCTestCase { change2.shouldNotBeNil() change2?.fromStatus.shouldEqual(.up) change2?.toStatus.shouldEqual(.down) - "\(change2!)".shouldContain("fromStatus: up, toStatus: down)") + "\(change2!)".shouldContain("1001 :: [ up] -> [ down]") membership.mark(member.node, as: .joining).shouldBeNil() // can't move "back" membership.mark(member.node, as: .up).shouldBeNil() // can't move "back", from down From 32a551627e855f903e0137e54e732b8e92d46885 Mon Sep 17 00:00:00 2001 From: Konrad `ktoso` Malawski Date: Fri, 1 May 2020 19:58:11 +0900 Subject: [PATCH 18/18] include time information in pprints so we can easier debug timing issues on CI --- .../Cluster/Cluster+Event.swift | 2 +- Sources/DistributedActors/utils.swift | 31 ++++++++++++++++--- .../ClusterLeaderActionsClusteredTests.swift | 15 +++++++-- .../DowningClusteredTests.swift | 2 +- 4 files changed, 42 insertions(+), 8 deletions(-) diff --git a/Sources/DistributedActors/Cluster/Cluster+Event.swift b/Sources/DistributedActors/Cluster/Cluster+Event.swift index 7bcac0cfd..77596f192 100644 --- a/Sources/DistributedActors/Cluster/Cluster+Event.swift +++ b/Sources/DistributedActors/Cluster/Cluster+Event.swift @@ -150,7 +150,7 @@ extension Cluster.MembershipChange: CustomStringConvertible { if let replaced = self.replaced { base = "[replaced:\(reflecting: replaced)] by \(reflecting: self.node)" } else { - base = "\(reflecting: self.node)" + base = "\(self.node)" } return base + " :: " + diff --git a/Sources/DistributedActors/utils.swift b/Sources/DistributedActors/utils.swift index a0d8822b5..2642ac552 100644 --- a/Sources/DistributedActors/utils.swift +++ b/Sources/DistributedActors/utils.swift @@ -72,24 +72,47 @@ internal func assertBacktrace(_ condition: @autoclosure () -> Bool, _ message: @ assert(condition(), { () in sact_dump_backtrace(); return message() }(), file: file, line: line) } +private func _createTimeFormatter() -> DateFormatter { + let formatter = DateFormatter() + formatter.dateFormat = "H:m:ss.SSSS" + formatter.locale = Locale(identifier: "en_US") + formatter.calendar = Calendar(identifier: .gregorian) + return formatter +} + /// Short for "pretty print", useful for debug tracing public func pprint(_ message: String, file: StaticString = #file, line: UInt = #line) { - print("[pprint][\(file):\(line)][\(_hackyPThreadThreadId())]: \(message)") - // print("[pprint][\(file):\(line)]: \(message)") + print(""" + [pprint]\ + [\(_createTimeFormatter().string(from: Date()))] \ + [\(file):\(line)]\ + [\(_hackyPThreadThreadId())]: \ + \(message) + """) } /// Like [pprint] but yellow, use for things that are better not to miss. public func pnote(_ message: String, file: StaticString = #file, line: UInt = #line) { let yellow = "\u{001B}[0;33m" let reset = "\u{001B}[0;0m" - print("\(yellow)\(file):\(line) : \(message)\(reset)") + print(""" + \(yellow)\ + [\(_createTimeFormatter().string(from: Date()))] \ + \(file):\(line) : \(message)\ + \(reset) + """) } /// Like [pprint] but green, use for notable "good" output. public func pinfo(_ message: String, file: StaticString = #file, line: UInt = #line) { let green = "\u{001B}[0;32m" let reset = "\u{001B}[0;0m" - print("\(green)\(file):\(line) : \(message)\(reset)") + print(""" + \(green)\ + [\(_createTimeFormatter().string(from: Date()))] \ + \(file):\(line) : \(message)\ + \(reset) + """) } internal func _hackyPThreadThreadId() -> String { diff --git a/Tests/DistributedActorsTests/Cluster/ClusterLeaderActionsClusteredTests.swift b/Tests/DistributedActorsTests/Cluster/ClusterLeaderActionsClusteredTests.swift index 5427dd476..048cb3960 100644 --- a/Tests/DistributedActorsTests/Cluster/ClusterLeaderActionsClusteredTests.swift +++ b/Tests/DistributedActorsTests/Cluster/ClusterLeaderActionsClusteredTests.swift @@ -240,10 +240,21 @@ final class ClusterLeaderActionsClusteredTests: ClusteredNodesTestBase { try self.ensureNodes(.down, on: third, nodes: secondNode) // on the leader node, the other node noticed as up: - let eventsOnFirstSub = try p1.expectMessages(count: 9) - for event in eventsOnFirstSub { + var eventsOnFirstSub: [Cluster.Event] = [] + var downFound = false + while eventsOnFirstSub.count < 12, !downFound { + let event = try p1.expectMessage() pinfo("Captured event: \(event)") + eventsOnFirstSub.append(event) + + switch event { + case .membershipChange(let change) where change.toStatus.isDown: + downFound = true + default: + () + } } + eventsOnFirstSub.shouldContain(.snapshot(.empty)) eventsOnFirstSub.shouldContain(.membershipChange(.init(node: first.cluster.node, fromStatus: nil, toStatus: .joining))) eventsOnFirstSub.shouldContain(.membershipChange(.init(node: secondNode, fromStatus: nil, toStatus: .joining))) diff --git a/Tests/DistributedActorsTests/Cluster/DowningStrategy/DowningClusteredTests.swift b/Tests/DistributedActorsTests/Cluster/DowningStrategy/DowningClusteredTests.swift index d2d99809b..217ddc934 100644 --- a/Tests/DistributedActorsTests/Cluster/DowningStrategy/DowningClusteredTests.swift +++ b/Tests/DistributedActorsTests/Cluster/DowningStrategy/DowningClusteredTests.swift @@ -97,7 +97,7 @@ final class DowningClusteredTests: ClusteredNodesTestBase { pinfo("MembershipChange on \(on.cluster.node.node): \(change)") return .catchContinue(change) case .reachabilityChange(let change) where change.member.node == expectedDownNode: - pnote("ReachabilityChange on \(otherNotDownPairSystem.cluster.node.node) = \(change)") + pnote("ReachabilityChange on \(otherNotDownPairSystem.cluster.node.node): \(change)") return .ignore default: // pnote("Event on \(otherNotDownPairSystem.cluster.node.node) = \(event)")