diff --git a/actor/actor_ref.go b/actor/actor_ref.go index 5549f6af..d588752a 100644 --- a/actor/actor_ref.go +++ b/actor/actor_ref.go @@ -42,6 +42,9 @@ import ( // // - Address: The actor address. One can use the address with Remoting to // interact with the actor by sending messages. +// +// - IsSingleton: The actor is a singleton +// if the actor is a singleton then the actor is created once in the cluster type ActorRef struct { // name defines the actor Name name string @@ -49,6 +52,8 @@ type ActorRef struct { kind string // address defines the actor address address *address.Address + // isSingleton defines if the actor is a singleton + isSingleton bool } // Name represents the actor given name @@ -66,6 +71,12 @@ func (x ActorRef) Address() *address.Address { return x.address } +// IsSingleton returns true if the actor is a singleton +// otherwise it returns false when the actor is not a singleton or cluster is not enabled +func (x ActorRef) IsSingleton() bool { + return x.isSingleton +} + // Equals is a convenient method to compare two ActorRef func (x ActorRef) Equals(actor ActorRef) bool { return x.address.Equals(actor.address) @@ -73,16 +84,18 @@ func (x ActorRef) Equals(actor ActorRef) bool { func fromActorRef(actorRef *internalpb.ActorRef) ActorRef { return ActorRef{ - name: actorRef.GetActorAddress().GetName(), - kind: actorRef.GetActorType(), - address: address.From(actorRef.GetActorAddress()), + name: actorRef.GetActorAddress().GetName(), + kind: actorRef.GetActorType(), + address: address.From(actorRef.GetActorAddress()), + isSingleton: actorRef.GetIsSingleton(), } } func fromPID(pid *PID) ActorRef { return ActorRef{ - name: pid.Name(), - kind: types.TypeName(pid.Actor()), - address: pid.Address(), + name: pid.Name(), + kind: types.Name(pid.Actor()), + address: pid.Address(), + isSingleton: pid.IsSingleton(), } } diff --git a/actor/actor_ref_test.go b/actor/actor_ref_test.go index 0fbbe71d..18ab55a3 100644 --- a/actor/actor_ref_test.go +++ b/actor/actor_ref_test.go @@ -63,6 +63,6 @@ func TestActorRef(t *testing.T) { } actorRef := fromPID(pid) require.Equal(t, "name", actorRef.Name()) - require.Equal(t, types.TypeName(actor), actorRef.Kind()) + require.Equal(t, types.Name(actor), actorRef.Kind()) }) } diff --git a/actor/actor_system.go b/actor/actor_system.go index 054144e3..14530d1d 100644 --- a/actor/actor_system.go +++ b/actor/actor_system.go @@ -1,7 +1,7 @@ /* * MIT License * - * Copyright (c) 2022-2025 Arsene Tochemey Gandote + * Copyright (c2) 2022-2025 Arsene Tochemey Gandote * * Permission is hereby granted, free of charge, to any person obtaining a copy * of this software and associated documentation files (the "Software"), to deal @@ -103,6 +103,17 @@ type ActorSystem interface { // A router is a special type of actor that helps distribute messages of the same type over a set of actors, so that messages can be processed in parallel. // A single actor will only process one message at a time. SpawnRouter(ctx context.Context, poolSize int, routeesKind Actor, opts ...RouterOption) (*PID, error) + // SpawnSingleton creates a singleton actor in the system. + // + // A singleton actor is instantiated when cluster mode is enabled. + // A singleton actor like any other actor is created only once within the system and in the cluster. + // A singleton actor is created with the default supervisor strategy and directive. + // A singleton actor once created lives throughout the lifetime of the given actor system. + // + // The cluster singleton is automatically started on the oldest node in the cluster. + // If the oldest node leaves the cluster, the singleton is restarted on the new oldest node. + // This is useful for managing shared resources or coordinating tasks that should be handled by a single actor. + SpawnSingleton(ctx context.Context, name string, actor Actor) error // Kill stops a given actor in the system Kill(ctx context.Context, name string) error // ReSpawn recreates a given actor in the system @@ -121,10 +132,10 @@ type ActorSystem interface { // When the cluster mode is not enabled an actor not found error will be returned // One can always check whether cluster is enabled before calling this method or just use the ActorOf method. RemoteActor(ctx context.Context, actorName string) (addr *address.Address, err error) - // ActorOf returns an existing actor in the local system or in the cluster when clustering is enabled - // When cluster mode is activated, the PID will be nil. - // When remoting is enabled this method will return and error - // An actor not found error is return when the actor is not found. + // ActorOf retrieves an existing actor within the local system or across the cluster if clustering is enabled. + // + // If the actor is found locally, its PID is returned. If the actor resides on a remote host, its address is returned. + // If the actor is not found, an error of type "actor not found" is returned. ActorOf(ctx context.Context, actorName string) (addr *address.Address, pid *PID, err error) // InCluster states whether the actor system has started within a cluster of nodes InCluster() bool @@ -186,8 +197,8 @@ type ActorSystem interface { handleRemoteAsk(ctx context.Context, to *PID, message proto.Message, timeout time.Duration) (response proto.Message, err error) // handleRemoteTell handles an asynchronous message to an actor handleRemoteTell(ctx context.Context, to *PID, message proto.Message) error - // setActor sets actor in the actor system actors registry - broadcastActor(actor *PID) + // broadcastActor sets actor in the actor system actors registry + broadcastActor(actor *PID, singleton bool) // getPeerStateFromStore returns the peer state from the cluster store getPeerStateFromStore(address string) (*internalpb.PeerState, error) // removePeerStateFromStore removes the peer state from the cluster store @@ -205,6 +216,7 @@ type ActorSystem interface { getUserGuardian() *PID getDeathWatch() *PID getDeadletter() *PID + getSingletonManager() *PID } // ActorSystem represent a collection of actors on a given node @@ -278,12 +290,14 @@ type actorSystem struct { rebalancingQueue chan *internalpb.PeerState rebalancedNodes goset.Set[string] - rebalancer *PID - rootGuardian *PID - userGuardian *PID - systemGuardian *PID - deathWatch *PID - deadletters *PID + rebalancer *PID + rootGuardian *PID + userGuardian *PID + systemGuardian *PID + deathWatch *PID + deadletter *PID + singletonManager *PID + startedAt *atomic.Int64 rebalancing *atomic.Bool rebalanceLocker *sync.Mutex @@ -446,8 +460,9 @@ func (x *actorSystem) Start(ctx context.Context) error { AddError(x.spawnSystemGuardian(ctx)). AddError(x.spawnUserGuardian(ctx)). AddError(x.spawnRebalancer(ctx)). - AddError(x.spawnJanitor(ctx)). + AddError(x.spawnDeathWatch(ctx)). AddError(x.spawnDeadletter(ctx)). + AddError(x.spawnSingletonManager(ctx)). Error(); err != nil { return errorschain. New(errorschain.ReturnAll()). @@ -633,7 +648,7 @@ func (x *actorSystem) Spawn(ctx context.Context, name string, actor Actor, opts } // check some preconditions - if err := x.checkSpawnPreconditions(ctx, name, actor); err != nil { + if err := x.checkSpawnPreconditions(ctx, name, actor, false); err != nil { return nil, err } @@ -653,9 +668,10 @@ func (x *actorSystem) Spawn(ctx context.Context, name string, actor Actor, opts x.actorsCounter.Inc() // add the given actor to the tree and supervise it - _ = x.actors.AddNode(x.userGuardian, pid) + guardian := x.getUserGuardian() + _ = x.actors.AddNode(guardian, pid) x.actors.AddWatcher(pid, x.deathWatch) - x.broadcastActor(pid) + x.broadcastActor(pid, false) return pid, nil } @@ -670,7 +686,7 @@ func (x *actorSystem) SpawnNamedFromFunc(ctx context.Context, name string, recei actor := newFuncActor(name, receiveFunc, config) // check some preconditions - if err := x.checkSpawnPreconditions(ctx, name, actor); err != nil { + if err := x.checkSpawnPreconditions(ctx, name, actor, false); err != nil { return nil, err } @@ -691,7 +707,7 @@ func (x *actorSystem) SpawnNamedFromFunc(ctx context.Context, name string, recei x.actorsCounter.Inc() _ = x.actors.AddNode(x.userGuardian, pid) x.actors.AddWatcher(pid, x.deathWatch) - x.broadcastActor(pid) + x.broadcastActor(pid, false) return pid, nil } @@ -709,6 +725,60 @@ func (x *actorSystem) SpawnRouter(ctx context.Context, poolSize int, routeesKind return x.Spawn(ctx, routerName, router) } +// SpawnSingleton creates a singleton actor in the system. +// +// A singleton actor is instantiated when cluster mode is enabled. +// A singleton actor like any other actor is created only once within the system and in the cluster. +// A singleton actor is created with the default supervisor strategy and directive. +// A singleton actor once created lives throughout the lifetime of the given actor system. +// +// The cluster singleton is automatically started on the oldest node in the cluster. +// If the oldest node leaves the cluster, the singleton is restarted on the new oldest node. +// This is useful for managing shared resources or coordinating tasks that should be handled by a single actor. +func (x *actorSystem) SpawnSingleton(ctx context.Context, name string, actor Actor) error { + if !x.started.Load() { + return ErrActorSystemNotStarted + } + + if !x.InCluster() { + return ErrClusterDisabled + } + + cl := x.getCluster() + + // only create the singleton actor on the oldest node in the cluster + if !cl.IsLeader(ctx) { + return x.spawnSingletonOnLeader(ctx, cl, name, actor) + } + + // check some preconditions + if err := x.checkSpawnPreconditions(ctx, name, actor, true); err != nil { + return err + } + + pid, err := x.configPID(ctx, name, actor, + WithLongLived(), + withSingleton(), + WithSupervisor( + NewSupervisor( + WithStrategy(OneForOneStrategy), + WithDirective(PanicError{}, StopDirective), + WithDirective(InternalError{}, StopDirective), + WithDirective(&runtime.PanicNilError{}, StopDirective), + ), + )) + if err != nil { + return err + } + + x.actorsCounter.Inc() + // add the given actor to the tree and supervise it + _ = x.actors.AddNode(x.singletonManager, pid) + x.actors.AddWatcher(pid, x.deathWatch) + x.broadcastActor(pid, true) + return nil +} + // Kill stops a given actor in the system func (x *actorSystem) Kill(ctx context.Context, name string) error { if !x.started.Load() { @@ -826,10 +896,10 @@ func (x *actorSystem) PeerAddress() string { return "" } -// ActorOf returns an existing actor in the local system or in the cluster when clustering is enabled -// When cluster mode is activated, the PID will be nil. -// When remoting is enabled this method will return and error -// An actor not found error is return when the actor is not found. +// ActorOf retrieves an existing actor within the local system or across the cluster if clustering is enabled. +// +// If the actor is found locally, its PID is returned. If the actor resides on a remote host, its address is returned. +// If the actor is not found, an error of type "actor not found" is returned. func (x *actorSystem) ActorOf(ctx context.Context, actorName string) (addr *address.Address, pid *PID, err error) { x.locker.Lock() @@ -1190,6 +1260,13 @@ func (x *actorSystem) RemoteSpawn(ctx context.Context, request *connect.Request[ return nil, connect.NewError(connect.CodeInternal, err) } + if msg.GetIsSingleton() { + if err := x.SpawnSingleton(ctx, msg.GetActorName(), actor); err != nil { + logger.Errorf("failed to create actor=(%s) on [host=%s, port=%d]: reason: (%v)", msg.GetActorName(), msg.GetHost(), msg.GetPort(), err) + return nil, connect.NewError(connect.CodeInternal, err) + } + } + if _, err = x.Spawn(ctx, msg.GetActorName(), actor); err != nil { logger.Errorf("failed to create actor=(%s) on [host=%s, port=%d]: reason: (%v)", msg.GetActorName(), msg.GetHost(), msg.GetPort(), err) return nil, connect.NewError(connect.CodeInternal, err) @@ -1237,7 +1314,7 @@ func (x *actorSystem) GetKinds(_ context.Context, request *connect.Request[inter kinds := make([]string, len(x.clusterConfig.Kinds())) for i, kind := range x.clusterConfig.Kinds() { - kinds[i] = types.TypeName(kind) + kinds[i] = types.Name(kind) } return connect.NewResponse(&internalpb.GetKindsResponse{Kinds: kinds}), nil @@ -1286,14 +1363,22 @@ func (x *actorSystem) getDeathWatch() *PID { return janitor } -// getDeadletters returns the system deadletters actor +// getDeadletters returns the system deadletter actor func (x *actorSystem) getDeadletter() *PID { x.locker.Lock() - deadletters := x.deadletters + deadletters := x.deadletter x.locker.Unlock() return deadletters } +// getSingletonManager returns the system singleton manager +func (x *actorSystem) getSingletonManager() *PID { + x.locker.Lock() + singletonManager := x.singletonManager + x.locker.Unlock() + return singletonManager +} + func (x *actorSystem) completeRebalancing() { x.rebalancing.Store(false) } @@ -1322,11 +1407,12 @@ func (x *actorSystem) getPeerStateFromStore(address string) (*internalpb.PeerSta } // broadcastActor broadcast the newly (re)spawned actor into the cluster -func (x *actorSystem) broadcastActor(actor *PID) { +func (x *actorSystem) broadcastActor(actor *PID, singleton bool) { if x.clusterEnabled.Load() { x.wireActorsQueue <- &internalpb.ActorRef{ ActorAddress: actor.Address().Address, - ActorType: types.TypeName(actor.Actor()), + ActorType: types.Name(actor.Actor()), + IsSingleton: singleton, } } } @@ -1407,7 +1493,7 @@ func (x *actorSystem) enableClustering(ctx context.Context) error { x.rebalancingQueue = make(chan *internalpb.PeerState, 1) for _, kind := range x.clusterConfig.Kinds() { x.registry.Register(kind) - x.logger.Infof("cluster kind=(%s) registered", types.TypeName(kind)) + x.logger.Infof("cluster kind=(%s) registered", types.Name(kind)) } x.locker.Unlock() @@ -1529,9 +1615,9 @@ func (x *actorSystem) shutdown(ctx context.Context) error { ctx, cancel := context.WithTimeout(ctx, x.shutdownTimeout) defer cancel() - var actorNames []string + var actorRefs []ActorRef for _, actor := range x.Actors() { - actorNames = append(actorNames, actor.Name()) + actorRefs = append(actorRefs, fromPID(actor)) } if err := x.getRootGuardian().Shutdown(ctx); err != nil { @@ -1547,7 +1633,7 @@ func (x *actorSystem) shutdown(ctx context.Context) error { if err := errorschain. New(errorschain.ReturnFirst()). - AddError(x.shutdownCluster(ctx, actorNames)). + AddError(x.shutdownCluster(ctx, actorRefs)). AddError(x.shutdownRemoting(ctx)). Error(); err != nil { x.logger.Errorf("%s failed to shutdown: %w", x.name, err) @@ -1731,7 +1817,10 @@ func (x *actorSystem) processPeerState(ctx context.Context, peer *cluster.Peer) } x.logger.Debugf("peer (%s) actors count (%d)", peerAddress, len(peerState.GetActors())) - x.clusterStore.set(peerState) + if err := x.clusterStore.set(peerState); err != nil { + x.logger.Error(err) + return err + } x.logger.Infof("peer sync(%s) successfully processed", peerAddress) return nil } @@ -1766,6 +1855,11 @@ func (x *actorSystem) configPID(ctx context.Context, name string, actor Actor, o pidOpts = append(pidOpts, withSupervisor(spawnConfig.supervisor)) } + // define the actor as singleton when necessary + if spawnConfig.asSingleton { + pidOpts = append(pidOpts, asSingleton()) + } + // enable stash if x.stashEnabled { pidOpts = append(pidOpts, withStash()) @@ -1915,8 +2009,8 @@ func (x *actorSystem) spawnUserGuardian(ctx context.Context) error { return nil } -// spawnRebalancer creates the cluster rebalancer -func (x *actorSystem) spawnJanitor(ctx context.Context) error { +// spawnDeathWatch creates the deathWatch actor +func (x *actorSystem) spawnDeathWatch(ctx context.Context) error { var err error actorName := x.reservedName(deathWatchType) @@ -1972,11 +2066,11 @@ func (x *actorSystem) spawnRebalancer(ctx context.Context) error { return nil } -// spawnDeadletter creates the deadletters synthetic actor +// spawnDeadletter creates the deadletter synthetic actor func (x *actorSystem) spawnDeadletter(ctx context.Context) error { var err error actorName := x.reservedName(deadletterType) - x.deadletters, err = x.configPID(ctx, + x.deadletter, err = x.configPID(ctx, actorName, newDeadLetter(), WithSupervisor( @@ -1988,18 +2082,35 @@ func (x *actorSystem) spawnDeadletter(ctx context.Context) error { ), ) if err != nil { - return fmt.Errorf("actor=%s failed to start deadletters: %w", actorName, err) + return fmt.Errorf("actor=%s failed to start deadletter: %w", actorName, err) } - // the deadletters is a child actor of the system guardian - _ = x.actors.AddNode(x.systemGuardian, x.deadletters) + // the deadletter is a child actor of the system guardian + _ = x.actors.AddNode(x.systemGuardian, x.deadletter) return nil } // checkSpawnPreconditions make sure before an actor is created some pre-conditions are checks -func (x *actorSystem) checkSpawnPreconditions(ctx context.Context, actorName string, kind Actor) error { +func (x *actorSystem) checkSpawnPreconditions(ctx context.Context, actorName string, kind Actor, singleton bool) error { // check the existence of the actor given the kind prior to creating it if x.clusterEnabled.Load() { + // a singleton actor must only have one instance at a given time of its kind + // in the whole cluster + if singleton { + id, err := x.cluster.LookupKind(ctx, types.Name(kind)) + if err != nil { + return err + } + + if id != "" { + return ErrSingletonAlreadyExists + } + + return nil + } + + // here we make sure in cluster mode that the given actor is uniquely created + // by checking both its kind and identifier existed, err := x.cluster.GetActor(ctx, actorName) if err != nil { if errors.Is(err, cluster.ErrActorNotFound) { @@ -2008,7 +2119,7 @@ func (x *actorSystem) checkSpawnPreconditions(ctx context.Context, actorName str return err } - if existed.GetActorType() == types.TypeName(kind) { + if existed.GetActorType() == types.Name(kind) { return ErrActorAlreadyExists(actorName) } } @@ -2017,11 +2128,32 @@ func (x *actorSystem) checkSpawnPreconditions(ctx context.Context, actorName str } // cleanupCluster cleans up the cluster -func (x *actorSystem) cleanupCluster(ctx context.Context, actorNames []string) error { +func (x *actorSystem) cleanupCluster(ctx context.Context, actorRefs []ActorRef) error { eg, ctx := errgroup.WithContext(ctx) - for _, actorName := range actorNames { - actorName := actorName + + // Remove singleton actors from the cluster + if x.cluster.IsLeader(ctx) { + for _, actorRef := range actorRefs { + if actorRef.IsSingleton() { + actorRef := actorRef + eg.Go(func() error { + kind := actorRef.Kind() + if err := x.cluster.RemoveKind(ctx, kind); err != nil { + x.logger.Errorf("failed to remove [actor kind=%s] from cluster: %v", kind, err) + return err + } + x.logger.Infof("[actor kind=%s] removed from cluster", kind) + return nil + }) + } + } + } + + // Remove all actors from the cluster + for _, actorRef := range actorRefs { + actorRef := actorRef eg.Go(func() error { + actorName := actorRef.Name() if err := x.cluster.RemoveActor(ctx, actorName); err != nil { x.logger.Errorf("failed to remove [actor=%s] from cluster: %v", actorName, err) return err @@ -2030,6 +2162,7 @@ func (x *actorSystem) cleanupCluster(ctx context.Context, actorNames []string) e return nil }) } + return eg.Wait() } @@ -2045,19 +2178,19 @@ func (x *actorSystem) getSetDeadlettersCount(ctx context.Context) { // using the default ask timeout // note: no need to check for error because this call is internal message, _ := from.Ask(ctx, to, message, DefaultAskTimeout) - // cast the response received from the deadletters + // cast the response received from the deadletter deadlettersCount := message.(*internalpb.DeadlettersCount) // set the counter x.deadlettersCounter.Store(uint64(deadlettersCount.GetTotalCount())) } } -func (x *actorSystem) shutdownCluster(ctx context.Context, actorNames []string) error { +func (x *actorSystem) shutdownCluster(ctx context.Context, actorRefs []ActorRef) error { if x.clusterEnabled.Load() { if x.cluster != nil { if err := errorschain. New(errorschain.ReturnFirst()). - AddError(x.cleanupCluster(ctx, actorNames)). + AddError(x.cleanupCluster(ctx, actorRefs)). AddError(x.cluster.Stop(ctx)). Error(); err != nil { x.reset() diff --git a/actor/actor_system_test.go b/actor/actor_system_test.go index d4adc6e2..d497757a 100644 --- a/actor/actor_system_test.go +++ b/actor/actor_system_test.go @@ -26,6 +26,7 @@ package actors import ( "context" + "crypto/tls" "errors" "net" "strconv" @@ -425,7 +426,7 @@ func TestActorSystem(t *testing.T) { var items []*goaktpb.ActorRestarted for message := range consumer.Iterator() { payload := message.Payload() - // only listening to deadletters + // only listening to deadletter restarted, ok := payload.(*goaktpb.ActorRestarted) if ok { items = append(items, restarted) @@ -723,12 +724,8 @@ func TestActorSystem(t *testing.T) { // stop the actor after some time util.Pause(time.Second) - t.Cleanup( - func() { - err = newActorSystem.Stop(ctx) - assert.NoError(t, err) - }, - ) + err = newActorSystem.Stop(ctx) + require.NoError(t, err) }) t.Run("With LocalActor", func(t *testing.T) { ctx := context.TODO() @@ -908,7 +905,7 @@ func TestActorSystem(t *testing.T) { }, ) }) - t.Run("With deadletters subscription ", func(t *testing.T) { + t.Run("With deadletter subscription ", func(t *testing.T) { ctx := context.TODO() sys, _ := NewActorSystem("testSys", WithLogger(log.DiscardLogger)) @@ -933,7 +930,7 @@ func TestActorSystem(t *testing.T) { // wait a while util.Pause(time.Second) - // every message sent to the actor will result in deadletters + // every message sent to the actor will result in deadletter for i := 0; i < 5; i++ { require.NoError(t, Tell(ctx, actorRef, new(testpb.TestSend))) } @@ -943,7 +940,7 @@ func TestActorSystem(t *testing.T) { var items []*goaktpb.Deadletter for message := range consumer.Iterator() { payload := message.Payload() - // only listening to deadletters + // only listening to deadletter deadletter, ok := payload.(*goaktpb.Deadletter) if ok { items = append(items, deadletter) @@ -965,7 +962,7 @@ func TestActorSystem(t *testing.T) { err = sys.Stop(ctx) assert.NoError(t, err) }) - t.Run("With deadletters subscription when not started", func(t *testing.T) { + t.Run("With deadletter subscription when not started", func(t *testing.T) { sys, _ := NewActorSystem("testSys", WithLogger(log.DiscardLogger)) // create a deadletter subscriber @@ -973,7 +970,7 @@ func TestActorSystem(t *testing.T) { require.Error(t, err) require.Nil(t, consumer) }) - t.Run("With deadletters unsubscription when not started", func(t *testing.T) { + t.Run("With deadletter unsubscription when not started", func(t *testing.T) { ctx := context.TODO() sys, _ := NewActorSystem("testSys", WithLogger(log.DiscardLogger)) @@ -1384,12 +1381,8 @@ func TestActorSystem(t *testing.T) { err = sys.Register(ctx, &exchanger{}) require.NoError(t, err) - t.Cleanup( - func() { - err = sys.Stop(ctx) - assert.NoError(t, err) - }, - ) + err = sys.Stop(ctx) + assert.NoError(t, err) }) t.Run("With Register when actor system not started", func(t *testing.T) { ctx := context.TODO() @@ -1529,7 +1522,7 @@ func TestActorSystem(t *testing.T) { require.Nil(t, addr) // spawn the remote actor - err = remoting.RemoteSpawn(ctx, host, remotingPort, actorName, "actors.exchanger") + err = remoting.RemoteSpawn(ctx, host, remotingPort, actorName, "actors.exchanger", false) require.NoError(t, err) // re-fetching the address of the actor should return not nil address after start @@ -1581,7 +1574,7 @@ func TestActorSystem(t *testing.T) { assert.EqualValues(t, 1, pid.ProcessedCount()) require.True(t, pid.IsRunning()) - // every message sent to the actor will result in deadletters + // every message sent to the actor will result in deadletter counter := 0 for i := 1; i <= 5; i++ { require.NoError(t, Tell(ctx, pid, new(testpb.TestSend))) @@ -1844,4 +1837,62 @@ func TestActorSystem(t *testing.T) { assert.True(t, pid.IsRunning()) assert.NoError(t, actorSystem.Stop(ctx)) }) + t.Run("With invalid remote config address", func(t *testing.T) { + remotingPort := dynaport.Get(1)[0] + + logger := log.DiscardLogger + host := "256.256.256.256" + + newActorSystem, err := NewActorSystem( + "test", + WithPassivationDisabled(), + WithLogger(logger), + WithRemote(remote.NewConfig(host, remotingPort, remote.WithWriteTimeout(-1))), + ) + require.Error(t, err) + require.Nil(t, newActorSystem) + }) + t.Run("With invalid cluster config", func(t *testing.T) { + logger := log.DiscardLogger + host := "127.0.0.1" + + // mock the discovery provider + provider := new(testkit.Provider) + newActorSystem, err := NewActorSystem( + "test", + WithPassivation(passivateAfter), + WithLogger(logger), + WithRemote(remote.NewConfig(host, 2222)), + WithCluster( + NewClusterConfig(). + WithKinds(new(mockActor)). + WithPartitionCount(0). + WithReplicaCount(1). + WithPeersPort(-1). + WithWAL(t.TempDir()). + WithMinimumPeersQuorum(1). + WithDiscoveryPort(-1). + WithDiscovery(provider)), + ) + require.Error(t, err) + require.Nil(t, newActorSystem) + }) + t.Run("With invalid TLS config", func(t *testing.T) { + logger := log.DiscardLogger + host := "127.0.0.1" + + newActorSystem, err := NewActorSystem( + "test", + WithPassivation(passivateAfter), + WithLogger(logger), + WithRemote(remote.NewConfig(host, 2222)), + WithTLS(&TLSInfo{ + ClientTLS: &tls.Config{InsecureSkipVerify: true}, // nolint + ServerTLS: nil, + }), + ) + require.Error(t, err) + require.ErrorIs(t, err, ErrInvalidTLSConfiguration) + require.Nil(t, newActorSystem) + }) } diff --git a/actor/cluster_config.go b/actor/cluster_config.go index 4cf3d20a..d5261b8f 100644 --- a/actor/cluster_config.go +++ b/actor/cluster_config.go @@ -73,8 +73,8 @@ func NewClusterConfig() *ClusterConfig { } } -// WithPartitionCount sets the cluster config partition count -// Partition cound should be a prime number. +// WithPartitionCount sets the cluster config partition count. +// Partition count should be a prime number. // ref: https://medium.com/swlh/why-should-the-length-of-your-hash-table-be-a-prime-number-760ec65a75d1 func (x *ClusterConfig) WithPartitionCount(count uint64) *ClusterConfig { x.partitionCount = count diff --git a/actor/cluster_singleton.go b/actor/cluster_singleton.go new file mode 100644 index 00000000..55615602 --- /dev/null +++ b/actor/cluster_singleton.go @@ -0,0 +1,166 @@ +/* + * MIT License + * + * Copyright (c) 2022-2025 Arsene Tochemey Gandote + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package actors + +import ( + "context" + "errors" + "fmt" + "runtime" + "time" + + "github.com/flowchartsman/retry" + + "github.com/tochemey/goakt/v3/goaktpb" + "github.com/tochemey/goakt/v3/internal/cluster" + "github.com/tochemey/goakt/v3/internal/internalpb" + "github.com/tochemey/goakt/v3/internal/types" + "github.com/tochemey/goakt/v3/log" +) + +// clusterSingletonManager is a system actor that manages the lifecycle of singleton actors +// in the cluster. This actor must be started when cluster mode is enabled in all nodes +// before any singleton actor is created. +type clusterSingletonManager struct { + logger log.Logger + pid *PID + cluster cluster.Interface +} + +// ensure clusterSingleton implements the Actor interface +var _ Actor = (*clusterSingletonManager)(nil) + +// newClusterSingletonManager creates a new cluster singleton actor. +func newClusterSingletonManager() Actor { + return &clusterSingletonManager{} +} + +// PreStart implements the pre-start hook. +func (x *clusterSingletonManager) PreStart(context.Context) error { + return nil +} + +// Receive handles messages received by the cluster singleton. +func (x *clusterSingletonManager) Receive(ctx *ReceiveContext) { + switch ctx.Message().(type) { + case *goaktpb.PostStart: + x.handlePostStart(ctx) + default: + ctx.Unhandled() + } +} + +// PostStop implements the post-stop hook. +func (x *clusterSingletonManager) PostStop(context.Context) error { + x.logger.Infof("%s stopped successfully", x.pid.Name()) + return nil +} + +// handlePostStart handles PostStart message +func (x *clusterSingletonManager) handlePostStart(ctx *ReceiveContext) { + x.pid = ctx.Self() + x.logger = ctx.Logger() + x.cluster = ctx.ActorSystem().getCluster() + x.logger.Infof("%s started successfully", x.pid.Name()) +} + +// spawnSingletonManager creates the singleton manager actor +// this is a system actor that manages the lifecycle of singleton actors +// in the cluster. This actor must be started when cluster mode is enabled in all nodes +// before any singleton actor is created. +func (x *actorSystem) spawnSingletonManager(ctx context.Context) error { + // only start the singleton manager when clustering is enabled + if !x.clusterEnabled.Load() { + return nil + } + + actorName := x.reservedName(singletonManagerType) + x.singletonManager, _ = x.configPID(ctx, + actorName, + newClusterSingletonManager(), + WithSupervisor( + NewSupervisor( + WithStrategy(OneForOneStrategy), + WithDirective(PanicError{}, RestartDirective), + WithDirective(InternalError{}, RestartDirective), + WithDirective(&runtime.PanicNilError{}, RestartDirective), + ), + ), + ) + + // the singletonManager is a child actor of the system guardian + _ = x.actors.AddNode(x.systemGuardian, x.singletonManager) + return nil +} + +func (x *actorSystem) spawnSingletonOnLeader(ctx context.Context, cl cluster.Interface, name string, actor Actor) error { + peers, err := cl.Peers(ctx) + if err != nil { + return fmt.Errorf("failed to spawn singleton actor: %w", err) + } + + // find the oldest node in the cluster + var leader *cluster.Peer + for _, peer := range peers { + if peer.Coordinator { + leader = peer + break + } + } + + // TODO: instead of using the cache add a method in the cluster to fetch peer info + if leader == nil { + return ErrLeaderNotFound + } + + var peerState *internalpb.PeerState + + // this is expected to be quick + retrier := retry.NewRetrier(3, 100*time.Millisecond, 300*time.Millisecond) + err = retrier.RunContext(ctx, func(_ context.Context) error { + peerState, err = x.getPeerStateFromStore(leader.PeerAddress()) + if err != nil { + if errors.Is(err, ErrPeerNotFound) { + return err + } + + // here we stop the retry because there is an error + return retry.Stop(err) + } + return nil + }) + + if err != nil { + return fmt.Errorf("failed to spawn singleton actor: %w", err) + } + + var ( + actorType = types.Name(actor) + host = peerState.GetHost() + port = int(peerState.GetRemotingPort()) + ) + + return x.remoting.RemoteSpawn(ctx, host, port, name, actorType, true) +} diff --git a/actor/cluster_singleton_test.go b/actor/cluster_singleton_test.go new file mode 100644 index 00000000..c1608e05 --- /dev/null +++ b/actor/cluster_singleton_test.go @@ -0,0 +1,163 @@ +/* + * MIT License + * + * Copyright (c) 2022-2025 Arsene Tochemey Gandote + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package actors + +import ( + "context" + "strings" + "testing" + "time" + + "github.com/stretchr/testify/require" + "github.com/travisjeffery/go-dynaport" + + "github.com/tochemey/goakt/v3/internal/util" + "github.com/tochemey/goakt/v3/log" + "github.com/tochemey/goakt/v3/remote" +) + +func TestSingletonActor(t *testing.T) { + t.Run("With Singleton Actor", func(t *testing.T) { + // create a context + ctx := context.TODO() + // start the NATS server + srv := startNatsServer(t) + + cl1, sd1 := startClusterSystem(t, srv.Addr().String()) + require.NotNil(t, cl1) + require.NotNil(t, sd1) + + cl2, sd2 := startClusterSystem(t, srv.Addr().String()) + require.NotNil(t, cl2) + require.NotNil(t, sd2) + + cl3, sd3 := startClusterSystem(t, srv.Addr().String()) + require.NotNil(t, cl3) + require.NotNil(t, sd3) + + util.Pause(time.Second) + + // create a singleton actor + actor := newMockActor() + actorName := "actorID" + // create a singleton actor + err := cl1.SpawnSingleton(ctx, actorName, actor) + require.NoError(t, err) + + // attempt to create another singleton actor with the same actor + err = cl2.SpawnSingleton(ctx, "actorName", actor) + require.Error(t, err) + require.Contains(t, err.Error(), ErrSingletonAlreadyExists.Error()) + + // free resources + require.NoError(t, cl3.Stop(ctx)) + require.NoError(t, sd3.Close()) + require.NoError(t, cl1.Stop(ctx)) + require.NoError(t, sd1.Close()) + require.NoError(t, cl2.Stop(ctx)) + require.NoError(t, sd2.Close()) + // shutdown the nats server gracefully + srv.Shutdown() + }) + t.Run("With Singleton Actor when cluster is not enabled returns error", func(t *testing.T) { + ctx := context.TODO() + remotingPort := dynaport.Get(1)[0] + + logger := log.DiscardLogger + host := "127.0.0.1" + + newActorSystem, err := NewActorSystem( + "test", + WithPassivationDisabled(), + WithLogger(logger), + WithRemote(remote.NewConfig(host, remotingPort)), + ) + require.NoError(t, err) + + // start the actor system + err = newActorSystem.Start(ctx) + require.NoError(t, err) + + util.Pause(time.Second) + + // create a singleton actor + actor := newMockActor() + actorName := "actorID" + // create a singleton actor + err = newActorSystem.SpawnSingleton(ctx, actorName, actor) + require.Error(t, err) + require.EqualError(t, err, ErrClusterDisabled.Error()) + + err = newActorSystem.Stop(ctx) + require.NoError(t, err) + }) + t.Run("With Singleton Actor when creating actor fails returns error", func(t *testing.T) { + // create a context + ctx := context.TODO() + // start the NATS server + srv := startNatsServer(t) + + cl1, sd1 := startClusterSystem(t, srv.Addr().String()) + require.NotNil(t, cl1) + require.NotNil(t, sd1) + + // create a singleton actor + actor := newMockActor() + actorName := strings.Repeat("a", 256) + err := cl1.SpawnSingleton(ctx, actorName, actor) + require.Error(t, err) + + // free resources + require.NoError(t, cl1.Stop(ctx)) + require.NoError(t, sd1.Close()) + // shutdown the nats server gracefully + srv.Shutdown() + }) + t.Run("With Singleton Actor when not started returns error", func(t *testing.T) { + ctx := context.TODO() + remotingPort := dynaport.Get(1)[0] + + logger := log.DiscardLogger + host := "127.0.0.1" + + newActorSystem, err := NewActorSystem( + "test", + WithPassivationDisabled(), + WithLogger(logger), + WithRemote(remote.NewConfig(host, remotingPort)), + ) + require.NoError(t, err) + + require.False(t, newActorSystem.Running()) + + // create a singleton actor + actor := newMockActor() + actorName := "actorID" + // create a singleton actor + err = newActorSystem.SpawnSingleton(ctx, actorName, actor) + require.Error(t, err) + require.EqualError(t, err, ErrActorSystemNotStarted.Error()) + }) +} diff --git a/actor/dead_letter.go b/actor/dead_letter.go index d29d6b5d..40f0e06a 100644 --- a/actor/dead_letter.go +++ b/actor/dead_letter.go @@ -37,7 +37,7 @@ import ( "github.com/tochemey/goakt/v3/log" ) -// deadletters is a synthetic actor that houses all deadletter +// deadletter is a synthetic actor that houses all deadletter // in GoAkt deadletter are messages that have not been handled type deadLetter struct { eventsStream *eventstream.EventsStream diff --git a/actor/dead_letter_test.go b/actor/dead_letter_test.go index b0ae683e..463c4514 100644 --- a/actor/dead_letter_test.go +++ b/actor/dead_letter_test.go @@ -65,7 +65,7 @@ func TestDeadletter(t *testing.T) { // wait a while util.Pause(time.Second) - // every message sent to the actor will result in deadletters + // every message sent to the actor will result in deadletter for i := 0; i < 5; i++ { require.NoError(t, Tell(ctx, actorRef, new(testpb.TestSend))) } @@ -75,7 +75,7 @@ func TestDeadletter(t *testing.T) { var items []*goaktpb.Deadletter for message := range consumer.Iterator() { payload := message.Payload() - // only listening to deadletters + // only listening to deadletter deadletter, ok := payload.(*goaktpb.Deadletter) if ok { items = append(items, deadletter) @@ -94,12 +94,8 @@ func TestDeadletter(t *testing.T) { err = sys.Unsubscribe(consumer) require.NoError(t, err) - t.Cleanup( - func() { - err = sys.Stop(ctx) - assert.NoError(t, err) - }, - ) + err = sys.Stop(ctx) + assert.NoError(t, err) }) t.Run("With GetDeadletters", func(t *testing.T) { ctx := context.TODO() @@ -126,7 +122,7 @@ func TestDeadletter(t *testing.T) { // wait a while util.Pause(time.Second) - // every message sent to the actor will result in deadletters + // every message sent to the actor will result in deadletter for i := 0; i < 5; i++ { require.NoError(t, Tell(ctx, actorRef, new(testpb.TestSend))) } @@ -136,7 +132,7 @@ func TestDeadletter(t *testing.T) { var items []*goaktpb.Deadletter for message := range consumer.Iterator() { payload := message.Payload() - // only listening to deadletters + // only listening to deadletter deadletter, ok := payload.(*goaktpb.Deadletter) if ok { items = append(items, deadletter) @@ -149,7 +145,7 @@ func TestDeadletter(t *testing.T) { for message := range consumer.Iterator() { payload := message.Payload() - // only listening to deadletters + // only listening to deadletter deadletter, ok := payload.(*goaktpb.Deadletter) if ok { items = append(items, deadletter) @@ -161,11 +157,7 @@ func TestDeadletter(t *testing.T) { err = sys.Unsubscribe(consumer) require.NoError(t, err) - t.Cleanup( - func() { - err = sys.Stop(ctx) - assert.NoError(t, err) - }, - ) + err = sys.Stop(ctx) + assert.NoError(t, err) }) } diff --git a/actor/errors.go b/actor/errors.go index 23b9cb3b..9f73a486 100644 --- a/actor/errors.go +++ b/actor/errors.go @@ -93,8 +93,12 @@ var ( ErrPriorityMessageRequired = errors.New("priority message type is required") // ErrActorAlreadyExists is returned when trying to create the same actor more than once ErrActorAlreadyExists = func(actorName string) error { return fmt.Errorf("actor=(%s) already exists", actorName) } - // ErrInvalidTLSConfiguration is returned whent the TLS configuration is not properly set + // ErrInvalidTLSConfiguration is returned when the TLS configuration is not properly set ErrInvalidTLSConfiguration = errors.New("TLS configuration is invalid") + // ErrSingletonAlreadyExists is returned when a given singleton actor type already exists + ErrSingletonAlreadyExists = errors.New("singleton already exists") + // ErrLeaderNotFound is returned when the cluster oldest node(leader) is not found + ErrLeaderNotFound = errors.New("leader is not found") ) // eof returns true if the given error is an EOF error diff --git a/actor/metric.go b/actor/metric.go index 498a1cf0..bd9bff8e 100644 --- a/actor/metric.go +++ b/actor/metric.go @@ -28,7 +28,7 @@ import "time" // Metric defines the actor system metric type Metric struct { - // DeadlettersCount returns the total number of deadletters + // DeadlettersCount returns the total number of deadletter deadlettersCount int64 // ActorsCount returns the total number of actors in the system actorsCount int64 @@ -36,7 +36,7 @@ type Metric struct { uptime int64 } -// DeadlettersCount returns the total number of deadletters +// DeadlettersCount returns the total number of deadletter func (m Metric) DeadlettersCount() int64 { return m.deadlettersCount } @@ -54,7 +54,7 @@ func (m Metric) Uptime() int64 { // ActorMetric defines actor specific metrics type ActorMetric struct { - // DeadlettersCount returns the total number of deadletters + // DeadlettersCount returns the total number of deadletter deadlettersCount uint64 // ActorsCount returns the total number of child actor given a specific PID childrenCount uint64 @@ -80,7 +80,7 @@ func (x ActorMetric) RestartCount() uint64 { return x.restartCount } -// DeadlettersCount returns the total number of deadletters +// DeadlettersCount returns the total number of deadletter func (x ActorMetric) DeadlettersCount() uint64 { return x.deadlettersCount } diff --git a/actor/pid.go b/actor/pid.go index 50055899..b3398ec0 100644 --- a/actor/pid.go +++ b/actor/pid.go @@ -145,6 +145,7 @@ type PID struct { goScheduler *goScheduler startedAt *atomic.Int64 + isSingleton atomic.Bool } // newPID creates a new pid @@ -184,6 +185,7 @@ func newPID(ctx context.Context, address *address.Address, actor Actor, opts ... pid.initMaxRetries.Store(DefaultInitMaxRetries) pid.latestReceiveDuration.Store(0) + pid.isSingleton.Store(false) pid.started.Store(false) pid.stopping.Store(false) pid.suspended.Store(false) @@ -374,6 +376,11 @@ func (pid *PID) IsSuspended() bool { return pid.suspended.Load() } +// IsSingleton returns true when the actor is a singleton +func (pid *PID) IsSingleton() bool { + return pid.isSingleton.Load() +} + // ActorSystem returns the actor system func (pid *PID) ActorSystem() ActorSystem { pid.fieldsLocker.RLock() @@ -604,7 +611,7 @@ func (pid *PID) SpawnChild(ctx context.Context, name string, actor Actor, opts . } // set the actor in the given actor system registry - pid.ActorSystem().broadcastActor(cid) + pid.ActorSystem().broadcastActor(cid, false) return cid, nil } @@ -1324,6 +1331,7 @@ func (pid *PID) reset() { pid.suspended.Store(false) pid.supervisor.Reset() pid.mailbox.Dispose() + pid.isSingleton.Store(false) } // freeWatchers releases all the actors watching this actor @@ -1645,7 +1653,7 @@ func (pid *PID) notifyParent(err error) { } } -// toDeadletters sends message to deadletters synthetic actor +// toDeadletters sends message to deadletter synthetic actor func (pid *PID) toDeadletters(receiveCtx *ReceiveContext, err error) { // the message is lost if pid.eventsStream == nil { @@ -1666,11 +1674,11 @@ func (pid *PID) toDeadletters(receiveCtx *ReceiveContext, err error) { sender = receiveCtx.Sender().Address().Address } - // get the deadletters synthetic actor and send a message to it + // get the deadletter synthetic actor and send a message to it receiver := pid.Address().Address - deadletters := pid.ActorSystem().getDeadletter() + deadletter := pid.ActorSystem().getDeadletter() _ = pid.Tell(context.Background(), - deadletters, + deadletter, &internalpb.EmitDeadletter{ Deadletter: &goaktpb.Deadletter{ Sender: sender, @@ -1854,7 +1862,7 @@ func (pid *PID) suspend(reason string) { }) } -// getDeadlettersCount gets deadletters +// getDeadlettersCount gets deadletter func (pid *PID) getDeadlettersCount(ctx context.Context) int64 { var ( name = pid.Name() @@ -1869,7 +1877,7 @@ func (pid *PID) getDeadlettersCount(ctx context.Context) int64 { // using the default ask timeout // note: no need to check for error because this call is internal message, _ := from.Ask(ctx, to, message, DefaultAskTimeout) - // cast the response received from the deadletters + // cast the response received from the deadletter deadlettersCount := message.(*internalpb.DeadlettersCount) return deadlettersCount.GetTotalCount() } diff --git a/actor/pid_option.go b/actor/pid_option.go index 509cc9f9..df3cf627 100644 --- a/actor/pid_option.go +++ b/actor/pid_option.go @@ -110,3 +110,10 @@ func withRemoting(remoting *Remoting) pidOption { pid.remoting = remoting } } + +// asSingleton set the actor as singleton +func asSingleton() pidOption { + return func(pid *PID) { + pid.isSingleton.Store(true) + } +} diff --git a/actor/pid_option_test.go b/actor/pid_option_test.go index 3574a72d..b36a91f0 100644 --- a/actor/pid_option_test.go +++ b/actor/pid_option_test.go @@ -97,6 +97,13 @@ func TestPIDOptions(t *testing.T) { option: withMailbox(mailbox), expected: &PID{mailbox: mailbox}, }, + { + name: "AsSingleton", + option: asSingleton(), + expected: &PID{ + isSingleton: atomicTrue, + }, + }, } for _, tc := range testCases { t.Run(tc.name, func(t *testing.T) { diff --git a/actor/rebalancer.go b/actor/rebalancer.go index bdccb2b3..d24c9dba 100644 --- a/actor/rebalancer.go +++ b/actor/rebalancer.go @@ -38,11 +38,6 @@ import ( "github.com/tochemey/goakt/v3/log" ) -type toRebalance struct { - name string - kind string -} - // rebalancer is a system actor that helps rebalance cluster // when the cluster topology changes type rebalancer struct { @@ -102,8 +97,9 @@ func (r *rebalancer) Rebalance(ctx *ReceiveContext) { if len(leaderShares) > 0 { eg.Go(func() error { for _, actor := range leaderShares { - if !isReservedName(actor.name) { - if err := r.recreateLocally(egCtx, actor); err != nil { + // never redistribute system actors + if !isReservedName(actor.GetActorName()) { + if err := r.recreateLocally(egCtx, actor, true); err != nil { return NewSpawnError(err) } } @@ -125,10 +121,10 @@ func (r *rebalancer) Rebalance(ctx *ReceiveContext) { } for _, actor := range actors { - // never redistribute system actors - if !isReservedName(actor.name) { + // never redistribute system actors and singleton actors + if !isReservedName(actor.GetActorName()) && !actor.GetIsSingleton() { if !peerFound { - err := r.recreateLocally(egCtx, actor) + err := r.recreateLocally(egCtx, actor, false) if err == nil { continue } @@ -138,8 +134,9 @@ func (r *rebalancer) Rebalance(ctx *ReceiveContext) { if err := r.remoting.RemoteSpawn(egCtx, peerState.GetHost(), int(peerState.GetRemotingPort()), - actor.name, - actor.kind); err != nil { + actor.GetActorName(), + actor.GetActorType(), + false); err != nil { logger.Error(err) return NewSpawnError(err) } @@ -175,37 +172,38 @@ func (r *rebalancer) PostStop(context.Context) error { } // computeRebalancing build the list of actors to create on the leader node and the peers in the cluster -func (r *rebalancer) computeRebalancing(totalPeers int, nodeLeftState *internalpb.PeerState) (leaderShares []*toRebalance, peersShares [][]*toRebalance) { - var ( - chunks [][]*toRebalance - actorsCount = len(nodeLeftState.GetActors()) - actors = nodeLeftState.GetActors() - ) - - // distribute actors amongst the peers with the leader taking the heavy load - switch { - case actorsCount < totalPeers: - for name, kind := range actors { - leaderShares = append(leaderShares, &toRebalance{ - name: name, - kind: kind, - }) - } - default: - toRebalances := make([]*toRebalance, 0, actorsCount) - for name, kind := range actors { - toRebalances = append(toRebalances, &toRebalance{ - name: name, - kind: kind, - }) - } - - quotient := actorsCount / totalPeers - remainder := actorsCount % totalPeers - leaderShares = toRebalances[:remainder] - chunks = slice.Chunk[*toRebalance](toRebalances[remainder:], quotient) +func (r *rebalancer) computeRebalancing(totalPeers int, nodeLeftState *internalpb.PeerState) (leaderShares []*internalpb.ActorProps, peersShares [][]*internalpb.ActorProps) { + actors := nodeLeftState.GetActors() + actorsCount := len(actors) + + // Collect all actors to be rebalanced + toRebalances := make([]*internalpb.ActorProps, 0, actorsCount) + for _, actorProps := range actors { + toRebalances = append(toRebalances, actorProps) } + // Separate singleton actors to be assigned to the leader + leaderShares = slice.Filter(toRebalances, func(actor *internalpb.ActorProps) bool { + return actor.GetIsSingleton() + }) + + // Remove singleton actors from the list + toRebalances = slice.Filter(toRebalances, func(actor *internalpb.ActorProps) bool { + return !actor.GetIsSingleton() + }) + + // Distribute remaining actors among peers + actorsCount = len(toRebalances) + quotient := actorsCount / totalPeers + remainder := actorsCount % totalPeers + + // Assign remainder actors to the leader + leaderShares = append(leaderShares, toRebalances[:remainder]...) + + // Chunk the remaining actors for peers + chunks := slice.Chunk(toRebalances[remainder:], quotient) + + // Ensure leader takes the first chunk if len(chunks) > 0 { leaderShares = append(leaderShares, chunks[0]...) } @@ -214,11 +212,17 @@ func (r *rebalancer) computeRebalancing(totalPeers int, nodeLeftState *internalp } // recreateLocally recreates the actor -func (r *rebalancer) recreateLocally(ctx context.Context, actor *toRebalance) error { - iactor, err := r.reflection.ActorFrom(actor.kind) +func (r *rebalancer) recreateLocally(ctx context.Context, actor *internalpb.ActorProps, enforceSingleton bool) error { + iactor, err := r.reflection.ActorFrom(actor.GetActorType()) if err != nil { return err } - _, err = r.pid.ActorSystem().Spawn(ctx, actor.name, iactor) + + if enforceSingleton && actor.GetIsSingleton() { + // spawn the singleton actor + return r.pid.ActorSystem().SpawnSingleton(ctx, actor.GetActorName(), iactor) + } + + _, err = r.pid.ActorSystem().Spawn(ctx, actor.GetActorName(), iactor) return err } diff --git a/actor/rebalancer_test.go b/actor/rebalancer_test.go index 46243a32..db93f3d8 100644 --- a/actor/rebalancer_test.go +++ b/actor/rebalancer_test.go @@ -40,159 +40,197 @@ import ( ) func TestRebalancing(t *testing.T) { - t.Run("With successful actors redeployment", func(t *testing.T) { - // create a context - ctx := context.TODO() - // start the NATS server - srv := startNatsServer(t) - - // create and start system cluster - node1, sd1 := startClusterSystem(t, srv.Addr().String()) - require.NotNil(t, node1) - require.NotNil(t, sd1) - - // create and start system cluster - node2, sd2 := startClusterSystem(t, srv.Addr().String()) - require.NotNil(t, node2) - require.NotNil(t, sd2) - - // create and start system cluster - node3, sd3 := startClusterSystem(t, srv.Addr().String()) - require.NotNil(t, node3) - require.NotNil(t, sd3) - - // let us create 4 actors on each node - for j := 1; j <= 4; j++ { - actorName := fmt.Sprintf("Node1-Actor-%d", j) - pid, err := node1.Spawn(ctx, actorName, newMockActor()) - require.NoError(t, err) - require.NotNil(t, pid) - } - - util.Pause(time.Second) - - for j := 1; j <= 4; j++ { - actorName := fmt.Sprintf("Node2-Actor-%d", j) - pid, err := node2.Spawn(ctx, actorName, newMockActor()) - require.NoError(t, err) - require.NotNil(t, pid) - } - - util.Pause(time.Second) - - for j := 1; j <= 4; j++ { - actorName := fmt.Sprintf("Node3-Actor-%d", j) - pid, err := node3.Spawn(ctx, actorName, newMockActor()) - require.NoError(t, err) - require.NotNil(t, pid) - } - - util.Pause(time.Second) - - // take down node2 - require.NoError(t, node2.Stop(ctx)) - require.NoError(t, sd2.Close()) - - // Wait for cluster rebalancing - util.Pause(time.Minute) - - sender, err := node1.LocalActor("Node1-Actor-1") + // create a context + ctx := context.TODO() + // start the NATS server + srv := startNatsServer(t) + + // create and start system cluster + node1, sd1 := startClusterSystem(t, srv.Addr().String()) + require.NotNil(t, node1) + require.NotNil(t, sd1) + + // create and start system cluster + node2, sd2 := startClusterSystem(t, srv.Addr().String()) + require.NotNil(t, node2) + require.NotNil(t, sd2) + + // create and start system cluster + node3, sd3 := startClusterSystem(t, srv.Addr().String()) + require.NotNil(t, node3) + require.NotNil(t, sd3) + + // let us create 4 actors on each node + for j := 1; j <= 4; j++ { + actorName := fmt.Sprintf("Node1-Actor-%d", j) + pid, err := node1.Spawn(ctx, actorName, newMockActor()) require.NoError(t, err) - require.NotNil(t, sender) + require.NotNil(t, pid) + } - // let us access some of the node2 actors from node 1 and node 3 - actorName := "Node2-Actor-1" - err = sender.SendAsync(ctx, actorName, new(testpb.TestSend)) + util.Pause(time.Second) + + for j := 1; j <= 4; j++ { + actorName := fmt.Sprintf("Node2-Actor-%d", j) + pid, err := node2.Spawn(ctx, actorName, newMockActor()) + require.NoError(t, err) + require.NotNil(t, pid) + } + + util.Pause(time.Second) + + for j := 1; j <= 4; j++ { + actorName := fmt.Sprintf("Node3-Actor-%d", j) + pid, err := node3.Spawn(ctx, actorName, newMockActor()) + require.NoError(t, err) + require.NotNil(t, pid) + } + + util.Pause(time.Second) + + // take down node2 + require.NoError(t, node2.Stop(ctx)) + require.NoError(t, sd2.Close()) + + // Wait for cluster rebalancing + util.Pause(time.Minute) + + sender, err := node1.LocalActor("Node1-Actor-1") + require.NoError(t, err) + require.NotNil(t, sender) + + // let us access some of the node2 actors from node 1 and node 3 + actorName := "Node2-Actor-1" + err = sender.SendAsync(ctx, actorName, new(testpb.TestSend)) + require.NoError(t, err) + + assert.NoError(t, node1.Stop(ctx)) + assert.NoError(t, node3.Stop(ctx)) + assert.NoError(t, sd1.Close()) + assert.NoError(t, sd3.Close()) + srv.Shutdown() +} + +func TestRebalancingWithTLSEnabled(t *testing.T) { + t.SkipNow() + // create a context + ctx := context.TODO() + // start the NATS server + srv := startNatsServer(t) + + // AutoGenerate TLS certs + conf := autotls.Config{ + AutoTLS: true, + ClientAuth: tls.RequireAndVerifyClientCert, + InsecureSkipVerify: true, + } + require.NoError(t, autotls.Setup(&conf)) + + // create and start system cluster + node1, sd1 := startClusterSystem(t, srv.Addr().String(), withTSL(conf)) + require.NotNil(t, node1) + require.NotNil(t, sd1) + + // create and start system cluster + node2, sd2 := startClusterSystem(t, srv.Addr().String(), withTSL(conf)) + require.NotNil(t, node2) + require.NotNil(t, sd2) + + // create and start system cluster + node3, sd3 := startClusterSystem(t, srv.Addr().String(), withTSL(conf)) + require.NotNil(t, node3) + require.NotNil(t, sd3) + + // let us create 4 actors on each node + for j := 1; j <= 4; j++ { + actorName := fmt.Sprintf("Node1-Actor-%d", j) + pid, err := node1.Spawn(ctx, actorName, newMockActor()) require.NoError(t, err) + require.NotNil(t, pid) + } + + util.Pause(time.Second) - t.Cleanup(func() { - assert.NoError(t, node1.Stop(ctx)) - assert.NoError(t, node3.Stop(ctx)) - assert.NoError(t, sd1.Close()) - assert.NoError(t, sd3.Close()) - srv.Shutdown() - }) - }) - t.Run("With successful actors redeployment when TLS enabled", func(t *testing.T) { - t.SkipNow() - // create a context - ctx := context.TODO() - // start the NATS server - srv := startNatsServer(t) - - // AutoGenerate TLS certs - conf := autotls.Config{ - AutoTLS: true, - ClientAuth: tls.RequireAndVerifyClientCert, - InsecureSkipVerify: true, - } - require.NoError(t, autotls.Setup(&conf)) - - // create and start system cluster - node1, sd1 := startClusterSystem(t, srv.Addr().String(), withTSL(conf)) - require.NotNil(t, node1) - require.NotNil(t, sd1) - - // create and start system cluster - node2, sd2 := startClusterSystem(t, srv.Addr().String(), withTSL(conf)) - require.NotNil(t, node2) - require.NotNil(t, sd2) - - // create and start system cluster - node3, sd3 := startClusterSystem(t, srv.Addr().String(), withTSL(conf)) - require.NotNil(t, node3) - require.NotNil(t, sd3) - - // let us create 4 actors on each node - for j := 1; j <= 4; j++ { - actorName := fmt.Sprintf("Node1-Actor-%d", j) - pid, err := node1.Spawn(ctx, actorName, newMockActor()) - require.NoError(t, err) - require.NotNil(t, pid) - } - - util.Pause(time.Second) - - for j := 1; j <= 4; j++ { - actorName := fmt.Sprintf("Node2-Actor-%d", j) - pid, err := node2.Spawn(ctx, actorName, newMockActor()) - require.NoError(t, err) - require.NotNil(t, pid) - } - - util.Pause(time.Second) - - for j := 1; j <= 4; j++ { - actorName := fmt.Sprintf("Node3-Actor-%d", j) - pid, err := node3.Spawn(ctx, actorName, newMockActor()) - require.NoError(t, err) - require.NotNil(t, pid) - } - - util.Pause(time.Second) - - // take down node2 - require.NoError(t, node2.Stop(ctx)) - require.NoError(t, sd2.Close()) - - // Wait for cluster rebalancing - util.Pause(time.Minute) - - sender, err := node1.LocalActor("Node1-Actor-1") + for j := 1; j <= 4; j++ { + actorName := fmt.Sprintf("Node2-Actor-%d", j) + pid, err := node2.Spawn(ctx, actorName, newMockActor()) require.NoError(t, err) - require.NotNil(t, sender) + require.NotNil(t, pid) + } - // let us access some of the node2 actors from node 1 and node 3 - actorName := "Node2-Actor-1" - err = sender.SendAsync(ctx, actorName, new(testpb.TestSend)) + util.Pause(time.Second) + + for j := 1; j <= 4; j++ { + actorName := fmt.Sprintf("Node3-Actor-%d", j) + pid, err := node3.Spawn(ctx, actorName, newMockActor()) require.NoError(t, err) + require.NotNil(t, pid) + } + + util.Pause(time.Second) + + // take down node2 + require.NoError(t, node2.Stop(ctx)) + require.NoError(t, sd2.Close()) + + // Wait for cluster rebalancing + util.Pause(time.Minute) + + sender, err := node1.LocalActor("Node1-Actor-1") + require.NoError(t, err) + require.NotNil(t, sender) + + // let us access some of the node2 actors from node 1 and node 3 + actorName := "Node2-Actor-1" + err = sender.SendAsync(ctx, actorName, new(testpb.TestSend)) + require.NoError(t, err) + + assert.NoError(t, node1.Stop(ctx)) + assert.NoError(t, node3.Stop(ctx)) + assert.NoError(t, sd1.Close()) + assert.NoError(t, sd3.Close()) + srv.Shutdown() +} + +func TestRebalancingWithSingletonActor(t *testing.T) { + // create a context + ctx := context.TODO() + // start the NATS server + srv := startNatsServer(t) + + // create and start system cluster + node1, sd1 := startClusterSystem(t, srv.Addr().String()) + require.NotNil(t, node1) + require.NotNil(t, sd1) + + // create and start system cluster + node2, sd2 := startClusterSystem(t, srv.Addr().String()) + require.NotNil(t, node2) + require.NotNil(t, sd2) + + // create and start system cluster + node3, sd3 := startClusterSystem(t, srv.Addr().String()) + require.NotNil(t, node3) + require.NotNil(t, sd3) + + // create a singleton actor + err := node1.SpawnSingleton(ctx, "actorName", newMockActor()) + require.NoError(t, err) + + util.Pause(time.Second) + + // take down node1 since it is the first node created in the cluster + require.NoError(t, node1.Stop(ctx)) + require.NoError(t, sd1.Close()) + + util.Pause(2 * time.Minute) + + _, _, err = node2.ActorOf(ctx, "actorName") + require.NoError(t, err) - t.Cleanup(func() { - assert.NoError(t, node1.Stop(ctx)) - assert.NoError(t, node3.Stop(ctx)) - assert.NoError(t, sd1.Close()) - assert.NoError(t, sd3.Close()) - srv.Shutdown() - }) - }) + assert.NoError(t, node2.Stop(ctx)) + assert.NoError(t, node3.Stop(ctx)) + assert.NoError(t, sd2.Close()) + assert.NoError(t, sd3.Close()) + srv.Shutdown() } diff --git a/actor/receive_context_test.go b/actor/receive_context_test.go index 19489445..e93ab2a9 100644 --- a/actor/receive_context_test.go +++ b/actor/receive_context_test.go @@ -1223,7 +1223,7 @@ func TestReceiveContext(t *testing.T) { self: pid1, } - // calling unhandled will push the current message to deadletters + // calling unhandled will push the current message to deadletter context.Unhandled() // wait for messages to be published @@ -1295,7 +1295,7 @@ func TestReceiveContext(t *testing.T) { self: pid1, } - // calling unhandled will push the current message to deadletters + // calling unhandled will push the current message to deadletter context.Unhandled() // wait for messages to be published @@ -1363,7 +1363,7 @@ func TestReceiveContext(t *testing.T) { self: pid1, } - // calling unhandled will push the current message to deadletters + // calling unhandled will push the current message to deadletter context.Unhandled() // wait for messages to be published diff --git a/actor/remoting.go b/actor/remoting.go index c2c71fec..e2b246d7 100644 --- a/actor/remoting.go +++ b/actor/remoting.go @@ -315,14 +315,15 @@ func (r *Remoting) RemoteBatchAsk(ctx context.Context, from, to *address.Address } // RemoteSpawn creates an actor on a remote node. The given actor needs to be registered on the remote node using the Register method of ActorSystem -func (r *Remoting) RemoteSpawn(ctx context.Context, host string, port int, name, actorType string) error { +func (r *Remoting) RemoteSpawn(ctx context.Context, host string, port int, name, actorType string, singleton bool) error { remoteClient := r.serviceClient(host, port) request := connect.NewRequest( &internalpb.RemoteSpawnRequest{ - Host: host, - Port: int32(port), - ActorName: name, - ActorType: actorType, + Host: host, + Port: int32(port), + ActorName: name, + ActorType: actorType, + IsSingleton: singleton, }, ) diff --git a/actor/remoting_test.go b/actor/remoting_test.go index 1df13398..d8a4bf8b 100644 --- a/actor/remoting_test.go +++ b/actor/remoting_test.go @@ -1696,7 +1696,7 @@ func TestRemotingSpawn(t *testing.T) { require.NoError(t, err) // spawn the remote actor - err = remoting.RemoteSpawn(ctx, host, remotingPort, actorName, "actors.exchanger") + err = remoting.RemoteSpawn(ctx, host, remotingPort, actorName, "actors.exchanger", false) require.NoError(t, err) // re-fetching the address of the actor should return not nil address after start @@ -1761,7 +1761,7 @@ func TestRemotingSpawn(t *testing.T) { require.Nil(t, addr) // spawn the remote actor - err = remoting.RemoteSpawn(ctx, sys.Host(), int(sys.Port()), actorName, "actors.exchanger") + err = remoting.RemoteSpawn(ctx, sys.Host(), int(sys.Port()), actorName, "actors.exchanger", false) require.Error(t, err) assert.EqualError(t, err, ErrTypeNotRegistered.Error()) @@ -1799,7 +1799,7 @@ func TestRemotingSpawn(t *testing.T) { actorName := uuid.NewString() remoting := NewRemoting() // spawn the remote actor - err = remoting.RemoteSpawn(ctx, host, remotingPort, actorName, "actors.exchanger") + err = remoting.RemoteSpawn(ctx, host, remotingPort, actorName, "actors.exchanger", false) require.Error(t, err) t.Cleanup( @@ -1862,7 +1862,7 @@ func TestRemotingSpawn(t *testing.T) { require.NoError(t, err) // spawn the remote actor - err = remoting.RemoteSpawn(ctx, host, remotingPort, actorName, "actors.exchanger") + err = remoting.RemoteSpawn(ctx, host, remotingPort, actorName, "actors.exchanger", false) require.NoError(t, err) // re-fetching the address of the actor should return not nil address after start diff --git a/actor/router.go b/actor/router.go index c244259b..54468b19 100644 --- a/actor/router.go +++ b/actor/router.go @@ -152,7 +152,7 @@ func (x *router) broadcast(ctx *ReceiveContext) { routees, proceed := x.availableRoutees() if !proceed { x.logger.Warn("no routees available. stopping.... Bye") - // push message to deadletters + // push message to deadletter ctx.Unhandled() // shutdown ctx.Shutdown() diff --git a/actor/router_test.go b/actor/router_test.go index 896a82ae..702bc4f8 100644 --- a/actor/router_test.go +++ b/actor/router_test.go @@ -246,7 +246,7 @@ func TestRouter(t *testing.T) { var items []*goaktpb.Deadletter for message := range consumer.Iterator() { payload := message.Payload() - // only listening to deadletters + // only listening to deadletter deadletter, ok := payload.(*goaktpb.Deadletter) if ok { items = append(items, deadletter) diff --git a/actor/spawn_option.go b/actor/spawn_option.go index c2a9c641..77a37ab9 100644 --- a/actor/spawn_option.go +++ b/actor/spawn_option.go @@ -38,6 +38,8 @@ type spawnConfig struct { // when the actor is passivated it is stopped which means it does not consume // any further resources like memory and cpu. The default value is 120 seconds passivateAfter *time.Duration + // specifies if the actor is a singleton + asSingleton bool } // newSpawnConfig creates an instance of spawnConfig @@ -110,3 +112,11 @@ func WithLongLived() SpawnOption { config.passivateAfter = &longLived }) } + +// withSingleton ensures that the actor is a singleton. +// This is an internal method to set the singleton flag +func withSingleton() SpawnOption { + return spawnOption(func(config *spawnConfig) { + config.asSingleton = true + }) +} diff --git a/actor/spawn_option_test.go b/actor/spawn_option_test.go index a1a0ea1b..2fc717f0 100644 --- a/actor/spawn_option_test.go +++ b/actor/spawn_option_test.go @@ -53,13 +53,19 @@ func TestSpawnOption(t *testing.T) { option.Apply(config) require.Equal(t, &spawnConfig{passivateAfter: &second}, config) }) - t.Run("spawn option with long-lived", func(t *testing.T) { config := &spawnConfig{} option := WithLongLived() option.Apply(config) require.Equal(t, &spawnConfig{passivateAfter: &longLived}, config) }) + + t.Run("spawn option with singleton", func(t *testing.T) { + config := &spawnConfig{} + option := withSingleton() + option.Apply(config) + require.Equal(t, &spawnConfig{asSingleton: true}, config) + }) } func TestNewSpawnConfig(t *testing.T) { diff --git a/actor/util.go b/actor/util.go index 251bfb8f..acca381c 100644 --- a/actor/util.go +++ b/actor/util.go @@ -65,6 +65,7 @@ const ( systemGuardianType deathWatchType deadletterType + singletonManagerType ) var ( @@ -77,12 +78,13 @@ var ( timers = timer.NewPool() systemNames = map[nameType]string{ - routerType: "GoAktRouter", - rebalancerType: "GoAktRebalancer", - rootGuardianType: "GoAktRootGuardian", - userGuardianType: "GoAktUserGuardian", - systemGuardianType: "GoAktSystemGuardian", - deathWatchType: "GoAktDeathWatch", - deadletterType: "GoAktDeadletter", + routerType: "GoAktRouter", + rebalancerType: "GoAktRebalancer", + rootGuardianType: "GoAktRootGuardian", + userGuardianType: "GoAktUserGuardian", + systemGuardianType: "GoAktSystemGuardian", + deathWatchType: "GoAktDeathWatch", + deadletterType: "GoAktDeadletter", + singletonManagerType: "GoAktSingletonManager", } ) diff --git a/client/client.go b/client/client.go index df151d98..8e8f2a59 100644 --- a/client/client.go +++ b/client/client.go @@ -133,12 +133,12 @@ func (x *Client) Kinds(ctx context.Context) ([]string, error) { } // Spawn creates an actor provided the actor name. -func (x *Client) Spawn(ctx context.Context, actor *Actor) (err error) { +func (x *Client) Spawn(ctx context.Context, actor *Actor, singleton bool) (err error) { x.locker.Lock() node := nextNode(x.balancer) x.locker.Unlock() remoteHost, remotePort := node.HostAndPort() - return node.Remoting().RemoteSpawn(ctx, remoteHost, remotePort, actor.Name(), actor.Kind()) + return node.Remoting().RemoteSpawn(ctx, remoteHost, remotePort, actor.Name(), actor.Kind(), singleton) } // SpawnWithBalancer creates an actor provided the actor name and the balancer strategy @@ -149,7 +149,7 @@ func (x *Client) SpawnWithBalancer(ctx context.Context, actor *Actor, strategy B node := nextNode(balancer) remoteHost, remotePort := node.HostAndPort() x.locker.Unlock() - return node.Remoting().RemoteSpawn(ctx, remoteHost, remotePort, actor.Name(), actor.Kind()) + return node.Remoting().RemoteSpawn(ctx, remoteHost, remotePort, actor.Name(), actor.Kind(), false) } // ReSpawn restarts a given actor diff --git a/client/client_test.go b/client/client_test.go index bea127e1..a9262c13 100644 --- a/client/client_test.go +++ b/client/client_test.go @@ -93,7 +93,7 @@ func TestClient(t *testing.T) { require.ElementsMatch(t, expected, kinds) actor := NewActor("client.testactor").WithName("actorName") - err = client.Spawn(ctx, actor) + err = client.Spawn(ctx, actor, false) require.NoError(t, err) util.Pause(time.Second) @@ -177,7 +177,7 @@ func TestClient(t *testing.T) { require.ElementsMatch(t, expected, kinds) actor := NewActor("client.testactor").WithName("actorName") - err = client.Spawn(ctx, actor) + err = client.Spawn(ctx, actor, false) require.NoError(t, err) util.Pause(time.Second) @@ -263,7 +263,7 @@ func TestClient(t *testing.T) { require.ElementsMatch(t, expected, kinds) actor := NewActor("client.testactor").WithName("actorName") - err = client.Spawn(ctx, actor) + err = client.Spawn(ctx, actor, false) require.NoError(t, err) util.Pause(time.Second) @@ -345,7 +345,7 @@ func TestClient(t *testing.T) { require.ElementsMatch(t, expected, kinds) actor := NewActor("client.testactor").WithName("actorName") - err = client.Spawn(ctx, actor) + err = client.Spawn(ctx, actor, false) require.NoError(t, err) util.Pause(time.Second) @@ -510,7 +510,7 @@ func TestClient(t *testing.T) { require.ElementsMatch(t, expected, kinds) actor := NewActor("client.testactor").WithName("actorName") - err = client.Spawn(ctx, actor) + err = client.Spawn(ctx, actor, false) require.NoError(t, err) util.Pause(time.Second) @@ -597,7 +597,7 @@ func TestClient(t *testing.T) { require.ElementsMatch(t, expected, kinds) actor := NewActor("client.testactor").WithName("actorName") - err = client.Spawn(ctx, actor) + err = client.Spawn(ctx, actor, false) require.NoError(t, err) util.Pause(time.Second) @@ -684,7 +684,7 @@ func TestClient(t *testing.T) { require.ElementsMatch(t, expected, kinds) actor := NewActor("client.testactor").WithName("actorName") - err = client.Spawn(ctx, actor) + err = client.Spawn(ctx, actor, false) require.NoError(t, err) util.Pause(time.Second) diff --git a/internal/cluster/engine.go b/internal/cluster/engine.go index bff3f82f..d50927e6 100644 --- a/internal/cluster/engine.go +++ b/internal/cluster/engine.go @@ -64,9 +64,10 @@ type EventType int const ( NodeJoined EventType = iota NodeLeft - actorsMap = "actors" - statesMap = "states" - jobKeysMap = "jobKeys" + actorsMap = "actors" + statesMap = "states" + jobKeysMap = "jobKeys" + actorKindsMap = "actorKinds" ) func (x EventType) String() string { @@ -102,11 +103,17 @@ type Interface interface { SetSchedulerJobKey(ctx context.Context, key string) error // SchedulerJobKeyExists checks the existence of a given key SchedulerJobKeyExists(ctx context.Context, key string) (bool, error) + // LookupKind checks the existence of a given actor kind in the cluster + // This function is mainly used when creating a singleton actor + LookupKind(ctx context.Context, kind string) (string, error) // UnsetSchedulerJobKey unsets the already set given key in the cluster UnsetSchedulerJobKey(ctx context.Context, key string) error // RemoveActor removes a given actor from the cluster. // An actor is removed from the cluster when this actor has been passivated. RemoveActor(ctx context.Context, actorName string) error + // RemoveKind removes a given actor from the cluster + // when the node where the kind is created shuts down + RemoveKind(ctx context.Context, kind string) error // Events returns a channel where cluster events are published Events() <-chan *Event // Peers returns a channel containing the list of peers at a given time @@ -148,10 +155,11 @@ type Engine struct { // this help set and fetch data from the Node client olric.Client - actorsMap olric.DMap - statesMap olric.DMap - jobKeysMap olric.DMap - kvStoreSize uint64 + actorsMap olric.DMap + statesMap olric.DMap + jobKeysMap olric.DMap + actorKindsMap olric.DMap + kvStoreSize uint64 // specifies the discovery node node *discovery.Node @@ -337,12 +345,18 @@ func (x *Engine) Start(ctx context.Context) error { return x.server.Shutdown(ctx) } + x.actorKindsMap, err = x.client.NewDMap(actorKindsMap) + if err != nil { + logger.Error(fmt.Errorf("failed to start the cluster Engine on node=(%s): %w", x.name, err)) + return x.server.Shutdown(ctx) + } + // set the peer state x.peerState = &internalpb.PeerState{ Host: x.node.Host, RemotingPort: int32(x.node.RemotingPort), PeersPort: int32(x.node.PeersPort), - Actors: map[string]string{}, + Actors: map[string]*internalpb.ActorProps{}, } if err := x.initializeState(ctx); err != nil { @@ -500,6 +514,17 @@ func (x *Engine) PutActor(ctx context.Context, actor *internalpb.ActorRef) error eg.Go(func() error { encoded, _ := encode(actor) key := actor.GetActorAddress().GetName() + if actor.GetIsSingleton() { + if err := errorschain. + New(errorschain.ReturnFirst()). + AddError(x.actorsMap.Put(ctx, key, encoded)). + AddError(x.actorKindsMap.Put(ctx, actor.GetActorType(), key)). + Error(); err != nil { + return fmt.Errorf("(%s) failed to sync actor=(%s): %v", x.node.PeersAddress(), actor.GetActorAddress().GetName(), err) + } + return nil + } + if err := x.actorsMap.Put(ctx, key, encoded); err != nil { return fmt.Errorf("(%s) failed to sync actor=(%s): %v", x.node.PeersAddress(), actor.GetActorAddress().GetName(), err) } @@ -508,7 +533,12 @@ func (x *Engine) PutActor(ctx context.Context, actor *internalpb.ActorRef) error eg.Go(func() error { actors := x.peerState.GetActors() - actors[actor.GetActorAddress().GetName()] = actor.GetActorType() + actorName := actor.GetActorAddress().GetName() + actors[actorName] = &internalpb.ActorProps{ + ActorName: actorName, + ActorType: actor.GetActorType(), + IsSingleton: actor.GetIsSingleton(), + } x.peerState.Actors = actors bytea, _ := proto.Marshal(x.peerState) @@ -637,6 +667,30 @@ func (x *Engine) RemoveActor(ctx context.Context, actorName string) error { return nil } +// RemoveKind removes a given actor from the cluster +// when the node where the kind is created shuts down +func (x *Engine) RemoveKind(ctx context.Context, kind string) error { + // return an error when the engine is not running + if !x.IsRunning() { + return ErrEngineNotRunning + } + + logger := x.logger + x.Lock() + defer x.Unlock() + + logger.Infof("removing actor kind (%s) from cluster", kind) + + _, err := x.actorKindsMap.Delete(ctx, kind) + if err != nil { + logger.Errorf("(%s) failed to remove actor kind=(%s) record from cluster: %v", x.node.PeersAddress(), kind, err) + return err + } + + logger.Infof("actor kind (%s) successfully removed from the cluster", kind) + return nil +} + // SetSchedulerJobKey sets a given key to the cluster func (x *Engine) SetSchedulerJobKey(ctx context.Context, key string) error { // return an error when the engine is not running @@ -663,6 +717,37 @@ func (x *Engine) SetSchedulerJobKey(ctx context.Context, key string) error { return nil } +// LookupKind checks the existence of a given actor kind in the cluster +// This function is mainly used when creating a singleton actor +func (x *Engine) LookupKind(ctx context.Context, kind string) (string, error) { + // return an error when the engine is not running + if !x.IsRunning() { + return "", ErrEngineNotRunning + } + + ctx, cancelFn := context.WithTimeout(ctx, x.readTimeout) + defer cancelFn() + + x.Lock() + defer x.Unlock() + + logger := x.logger + + logger.Infof("checking actor kind (%s) existence in the cluster", kind) + + resp, err := x.actorKindsMap.Get(ctx, kind) + if err != nil { + if errors.Is(err, olric.ErrKeyNotFound) { + logger.Warnf("actor kind=%s is not found in the cluster", kind) + return "", nil + } + + logger.Errorf("[%s] failed to check actor kind (%s) existence: %v", x.node.PeersAddress(), kind, err) + return "", err + } + return resp.String() +} + // SchedulerJobKeyExists checks the existence of a given key func (x *Engine) SchedulerJobKeyExists(ctx context.Context, key string) (bool, error) { // return an error when the engine is not running diff --git a/internal/cluster/engine_test.go b/internal/cluster/engine_test.go index ab65e278..7b14bb1e 100644 --- a/internal/cluster/engine_test.go +++ b/internal/cluster/engine_test.go @@ -194,7 +194,7 @@ func TestSingleNode(t *testing.T) { // mock the discovery provider provider := new(testkit.Provider) - provider.EXPECT().ID().Return("testDisco") + provider.EXPECT().ID().Return("id") provider.EXPECT().Initialize().Return(nil) provider.EXPECT().Register().Return(nil) provider.EXPECT().Deregister().Return(nil) @@ -377,6 +377,82 @@ func TestSingleNode(t *testing.T) { // stop the startNode require.NoError(t, cluster.Stop(ctx)) }) + t.Run("With Put/RemoveKind and LookupKind", func(t *testing.T) { + // create the context + ctx := context.TODO() + + // generate the ports for the single startNode + nodePorts := dynaport.Get(3) + discoveryPort := nodePorts[0] + clusterPort := nodePorts[1] + remotingPort := nodePorts[2] + + // define discovered addresses + addrs := []string{ + fmt.Sprintf("127.0.0.1:%d", discoveryPort), + } + + // mock the discovery provider + provider := new(testkit.Provider) + + provider.EXPECT().ID().Return("id") + provider.EXPECT().Initialize().Return(nil) + provider.EXPECT().Register().Return(nil) + provider.EXPECT().Deregister().Return(nil) + provider.EXPECT().DiscoverPeers().Return(addrs, nil) + provider.EXPECT().Close().Return(nil) + + // create a Node startNode + host := "127.0.0.1" + hostNode := discovery.Node{ + Name: host, + Host: host, + DiscoveryPort: discoveryPort, + PeersPort: clusterPort, + RemotingPort: remotingPort, + } + + cluster, err := NewEngine("test", provider, &hostNode, WithLogger(log.DiscardLogger)) + require.NotNil(t, cluster) + require.NoError(t, err) + + // start the Node startNode + err = cluster.Start(ctx) + require.NoError(t, err) + + // create an actor + actorName := uuid.NewString() + actorKind := "kind" + actor := &internalpb.ActorRef{ + ActorAddress: &goaktpb.Address{Name: actorName}, + ActorType: actorKind, + IsSingleton: true, + } + + // replicate the actor in the Node + err = cluster.PutActor(ctx, actor) + require.NoError(t, err) + + actual, err := cluster.LookupKind(ctx, actorKind) + require.NoError(t, err) + require.NotEmpty(t, actual) + + // remove the kind + err = cluster.RemoveKind(ctx, actorKind) + require.NoError(t, err) + + // check the kind existence + actual, err = cluster.LookupKind(ctx, actorKind) + require.NoError(t, err) + require.Empty(t, actual) + + // shutdown the Node startNode + util.Pause(time.Second) + + // stop the startNode + require.NoError(t, cluster.Stop(ctx)) + provider.AssertExpectations(t) + }) } func TestMultipleNodes(t *testing.T) { diff --git a/internal/collection/slice/filter.go b/internal/collection/slice/filter.go new file mode 100644 index 00000000..4fef0a43 --- /dev/null +++ b/internal/collection/slice/filter.go @@ -0,0 +1,36 @@ +/* + * MIT License + * + * Copyright (c) 2022-2025 Arsene Tochemey Gandote + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package slice + +// Filter returns a new slice containing elements from the input slice that satisfy the predicate. +func Filter[T any](slice []T, predicate func(T) bool) []T { + var result []T + for _, v := range slice { + if predicate(v) { + result = append(result, v) + } + } + return result +} diff --git a/internal/collection/slice/filter_test.go b/internal/collection/slice/filter_test.go new file mode 100644 index 00000000..151be6fd --- /dev/null +++ b/internal/collection/slice/filter_test.go @@ -0,0 +1,42 @@ +/* + * MIT License + * + * Copyright (c) 2022-2025 Arsene Tochemey Gandote + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package slice + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestFilter(t *testing.T) { + slice := []int{1, 2, 3, 4, 5, 6, 7, 8, 9} + result := Filter(slice, func(v int) bool { + return v%2 == 0 + }) + + expected := []int{2, 4, 6, 8} + require.Len(t, result, 4) + require.ElementsMatch(t, expected, result) +} diff --git a/internal/collection/syncmap/map.go b/internal/collection/syncmap/map.go index 36feb11c..e28977d3 100644 --- a/internal/collection/syncmap/map.go +++ b/internal/collection/syncmap/map.go @@ -102,3 +102,21 @@ func (s *Map[K, V]) Range(f func(K, V)) { f(k, v) } } + +// Values returns the values in the Map +func (s *Map[K, V]) Values() []V { + s.mu.RLock() + defer s.mu.RUnlock() + values := make([]V, 0, len(s.data)) + for _, v := range s.data { + values = append(values, v) + } + return values +} + +// Reset clears all key-value pairs from the Map. +func (s *Map[K, V]) Reset() { + s.mu.Lock() + clear(s.data) + s.mu.Unlock() +} diff --git a/internal/internalpb/actor.pb.go b/internal/internalpb/actor.pb.go index 23e64224..fbe09727 100644 --- a/internal/internalpb/actor.pb.go +++ b/internal/internalpb/actor.pb.go @@ -28,7 +28,9 @@ type ActorRef struct { // Specifies the actor address ActorAddress *goaktpb.Address `protobuf:"bytes,1,opt,name=actor_address,json=actorAddress,proto3" json:"actor_address,omitempty"` // Specifies the actor type - ActorType string `protobuf:"bytes,2,opt,name=actor_type,json=actorType,proto3" json:"actor_type,omitempty"` + ActorType string `protobuf:"bytes,2,opt,name=actor_type,json=actorType,proto3" json:"actor_type,omitempty"` + // Specifies if the actor is a singleton + IsSingleton bool `protobuf:"varint,3,opt,name=is_singleton,json=isSingleton,proto3" json:"is_singleton,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } @@ -77,30 +79,111 @@ func (x *ActorRef) GetActorType() string { return "" } +func (x *ActorRef) GetIsSingleton() bool { + if x != nil { + return x.IsSingleton + } + return false +} + +// ActorProps defines the properties of an actor +// that can be used to spawn an actor remotely. +type ActorProps struct { + state protoimpl.MessageState `protogen:"open.v1"` + // Specifies the actor name. + ActorName string `protobuf:"bytes,1,opt,name=actor_name,json=actorName,proto3" json:"actor_name,omitempty"` + // Specifies the actor type + ActorType string `protobuf:"bytes,2,opt,name=actor_type,json=actorType,proto3" json:"actor_type,omitempty"` + // Specifies if the actor is a singleton + IsSingleton bool `protobuf:"varint,3,opt,name=is_singleton,json=isSingleton,proto3" json:"is_singleton,omitempty"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *ActorProps) Reset() { + *x = ActorProps{} + mi := &file_internal_actor_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ActorProps) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ActorProps) ProtoMessage() {} + +func (x *ActorProps) ProtoReflect() protoreflect.Message { + mi := &file_internal_actor_proto_msgTypes[1] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ActorProps.ProtoReflect.Descriptor instead. +func (*ActorProps) Descriptor() ([]byte, []int) { + return file_internal_actor_proto_rawDescGZIP(), []int{1} +} + +func (x *ActorProps) GetActorName() string { + if x != nil { + return x.ActorName + } + return "" +} + +func (x *ActorProps) GetActorType() string { + if x != nil { + return x.ActorType + } + return "" +} + +func (x *ActorProps) GetIsSingleton() bool { + if x != nil { + return x.IsSingleton + } + return false +} + var File_internal_actor_proto protoreflect.FileDescriptor var file_internal_actor_proto_rawDesc = string([]byte{ 0x0a, 0x14, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x61, 0x63, 0x74, 0x6f, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0a, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x1a, 0x11, 0x67, 0x6f, 0x61, 0x6b, 0x74, 0x2f, 0x67, 0x6f, 0x61, 0x6b, 0x74, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x60, 0x0a, 0x08, 0x41, 0x63, 0x74, 0x6f, 0x72, 0x52, 0x65, - 0x66, 0x12, 0x35, 0x0a, 0x0d, 0x61, 0x63, 0x74, 0x6f, 0x72, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, - 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x67, 0x6f, 0x61, 0x6b, 0x74, - 0x70, 0x62, 0x2e, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x52, 0x0c, 0x61, 0x63, 0x74, 0x6f, - 0x72, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x61, 0x63, 0x74, 0x6f, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x83, 0x01, 0x0a, 0x08, 0x41, 0x63, 0x74, 0x6f, 0x72, 0x52, + 0x65, 0x66, 0x12, 0x35, 0x0a, 0x0d, 0x61, 0x63, 0x74, 0x6f, 0x72, 0x5f, 0x61, 0x64, 0x64, 0x72, + 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x67, 0x6f, 0x61, 0x6b, + 0x74, 0x70, 0x62, 0x2e, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x52, 0x0c, 0x61, 0x63, 0x74, + 0x6f, 0x72, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x61, 0x63, 0x74, + 0x6f, 0x72, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x61, + 0x63, 0x74, 0x6f, 0x72, 0x54, 0x79, 0x70, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x73, 0x5f, 0x73, + 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x74, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, + 0x69, 0x73, 0x53, 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x74, 0x6f, 0x6e, 0x22, 0x6d, 0x0a, 0x0a, 0x41, + 0x63, 0x74, 0x6f, 0x72, 0x50, 0x72, 0x6f, 0x70, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x61, 0x63, 0x74, + 0x6f, 0x72, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x61, + 0x63, 0x74, 0x6f, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x61, 0x63, 0x74, 0x6f, 0x72, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x61, 0x63, - 0x74, 0x6f, 0x72, 0x54, 0x79, 0x70, 0x65, 0x42, 0xa3, 0x01, 0x0a, 0x0e, 0x63, 0x6f, 0x6d, 0x2e, - 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x42, 0x0a, 0x41, 0x63, 0x74, 0x6f, - 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x48, 0x02, 0x50, 0x01, 0x5a, 0x3b, 0x67, 0x69, 0x74, 0x68, - 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x74, 0x6f, 0x63, 0x68, 0x65, 0x6d, 0x65, 0x79, 0x2f, - 0x67, 0x6f, 0x61, 0x6b, 0x74, 0x2f, 0x76, 0x33, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, - 0x6c, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x3b, 0x69, 0x6e, 0x74, - 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0xa2, 0x02, 0x03, 0x49, 0x58, 0x58, 0xaa, 0x02, 0x0a, - 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0xca, 0x02, 0x0a, 0x49, 0x6e, 0x74, - 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0xe2, 0x02, 0x16, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, - 0x61, 0x6c, 0x70, 0x62, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, - 0xea, 0x02, 0x0a, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x62, 0x06, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x74, 0x6f, 0x72, 0x54, 0x79, 0x70, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x73, 0x5f, 0x73, 0x69, + 0x6e, 0x67, 0x6c, 0x65, 0x74, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x69, + 0x73, 0x53, 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x74, 0x6f, 0x6e, 0x42, 0xa3, 0x01, 0x0a, 0x0e, 0x63, + 0x6f, 0x6d, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x42, 0x0a, 0x41, + 0x63, 0x74, 0x6f, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x48, 0x02, 0x50, 0x01, 0x5a, 0x3b, 0x67, + 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x74, 0x6f, 0x63, 0x68, 0x65, 0x6d, + 0x65, 0x79, 0x2f, 0x67, 0x6f, 0x61, 0x6b, 0x74, 0x2f, 0x76, 0x33, 0x2f, 0x69, 0x6e, 0x74, 0x65, + 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x3b, + 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0xa2, 0x02, 0x03, 0x49, 0x58, 0x58, + 0xaa, 0x02, 0x0a, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0xca, 0x02, 0x0a, + 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0xe2, 0x02, 0x16, 0x49, 0x6e, 0x74, + 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, + 0x61, 0x74, 0x61, 0xea, 0x02, 0x0a, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, + 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, }) var ( @@ -115,13 +198,14 @@ func file_internal_actor_proto_rawDescGZIP() []byte { return file_internal_actor_proto_rawDescData } -var file_internal_actor_proto_msgTypes = make([]protoimpl.MessageInfo, 1) +var file_internal_actor_proto_msgTypes = make([]protoimpl.MessageInfo, 2) var file_internal_actor_proto_goTypes = []any{ (*ActorRef)(nil), // 0: internalpb.ActorRef - (*goaktpb.Address)(nil), // 1: goaktpb.Address + (*ActorProps)(nil), // 1: internalpb.ActorProps + (*goaktpb.Address)(nil), // 2: goaktpb.Address } var file_internal_actor_proto_depIdxs = []int32{ - 1, // 0: internalpb.ActorRef.actor_address:type_name -> goaktpb.Address + 2, // 0: internalpb.ActorRef.actor_address:type_name -> goaktpb.Address 1, // [1:1] is the sub-list for method output_type 1, // [1:1] is the sub-list for method input_type 1, // [1:1] is the sub-list for extension type_name @@ -140,7 +224,7 @@ func file_internal_actor_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: unsafe.Slice(unsafe.StringData(file_internal_actor_proto_rawDesc), len(file_internal_actor_proto_rawDesc)), NumEnums: 0, - NumMessages: 1, + NumMessages: 2, NumExtensions: 0, NumServices: 0, }, diff --git a/internal/internalpb/peers.pb.go b/internal/internalpb/peers.pb.go index 7ad15c11..91bf408a 100644 --- a/internal/internalpb/peers.pb.go +++ b/internal/internalpb/peers.pb.go @@ -30,8 +30,8 @@ type PeerState struct { // Specifies the remoting host PeersPort int32 `protobuf:"varint,3,opt,name=peers_port,json=peersPort,proto3" json:"peers_port,omitempty"` // Specifies the list of actors - // actorName -> actorKind - Actors map[string]string `protobuf:"bytes,4,rep,name=actors,proto3" json:"actors,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` + // actorName -> ActorProps + Actors map[string]*ActorProps `protobuf:"bytes,4,rep,name=actors,proto3" json:"actors,omitempty" protobuf_key:"bytes,1,opt,name=key" protobuf_val:"bytes,2,opt,name=value"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } @@ -87,7 +87,7 @@ func (x *PeerState) GetPeersPort() int32 { return 0 } -func (x *PeerState) GetActors() map[string]string { +func (x *PeerState) GetActors() map[string]*ActorProps { if x != nil { return x.Actors } @@ -189,39 +189,42 @@ var File_internal_peers_proto protoreflect.FileDescriptor var file_internal_peers_proto_rawDesc = string([]byte{ 0x0a, 0x14, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0a, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, - 0x70, 0x62, 0x22, 0xd9, 0x01, 0x0a, 0x09, 0x50, 0x65, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, - 0x12, 0x12, 0x0a, 0x04, 0x68, 0x6f, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, - 0x68, 0x6f, 0x73, 0x74, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x69, 0x6e, 0x67, - 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0c, 0x72, 0x65, 0x6d, - 0x6f, 0x74, 0x69, 0x6e, 0x67, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x70, 0x65, 0x65, - 0x72, 0x73, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x70, - 0x65, 0x65, 0x72, 0x73, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x39, 0x0a, 0x06, 0x61, 0x63, 0x74, 0x6f, - 0x72, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, - 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x2e, - 0x41, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x61, 0x63, 0x74, - 0x6f, 0x72, 0x73, 0x1a, 0x39, 0x0a, 0x0b, 0x41, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x45, 0x6e, 0x74, - 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x41, - 0x0a, 0x09, 0x52, 0x65, 0x62, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x12, 0x34, 0x0a, 0x0a, 0x70, - 0x65, 0x65, 0x72, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x15, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, 0x50, 0x65, 0x65, - 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x09, 0x70, 0x65, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x22, 0x36, 0x0a, 0x11, 0x52, 0x65, 0x62, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x43, 0x6f, - 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x61, - 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x70, 0x65, - 0x65, 0x72, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x42, 0xa3, 0x01, 0x0a, 0x0e, 0x63, 0x6f, - 0x6d, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x42, 0x0a, 0x50, 0x65, - 0x65, 0x72, 0x73, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x48, 0x02, 0x50, 0x01, 0x5a, 0x3b, 0x67, 0x69, - 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x74, 0x6f, 0x63, 0x68, 0x65, 0x6d, 0x65, - 0x79, 0x2f, 0x67, 0x6f, 0x61, 0x6b, 0x74, 0x2f, 0x76, 0x33, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, - 0x6e, 0x61, 0x6c, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x3b, 0x69, - 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0xa2, 0x02, 0x03, 0x49, 0x58, 0x58, 0xaa, - 0x02, 0x0a, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0xca, 0x02, 0x0a, 0x49, - 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0xe2, 0x02, 0x16, 0x49, 0x6e, 0x74, 0x65, - 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, - 0x74, 0x61, 0xea, 0x02, 0x0a, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x62, - 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x70, 0x62, 0x1a, 0x14, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x61, 0x63, 0x74, + 0x6f, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xf1, 0x01, 0x0a, 0x09, 0x50, 0x65, 0x65, + 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x68, 0x6f, 0x73, 0x74, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x68, 0x6f, 0x73, 0x74, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, + 0x6d, 0x6f, 0x74, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x05, 0x52, 0x0c, 0x72, 0x65, 0x6d, 0x6f, 0x74, 0x69, 0x6e, 0x67, 0x50, 0x6f, 0x72, 0x74, 0x12, + 0x1d, 0x0a, 0x0a, 0x70, 0x65, 0x65, 0x72, 0x73, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x05, 0x52, 0x09, 0x70, 0x65, 0x65, 0x72, 0x73, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x39, + 0x0a, 0x06, 0x61, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, + 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, 0x50, 0x65, 0x65, 0x72, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x2e, 0x41, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x06, 0x61, 0x63, 0x74, 0x6f, 0x72, 0x73, 0x1a, 0x51, 0x0a, 0x0b, 0x41, 0x63, 0x74, + 0x6f, 0x72, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x2c, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x69, 0x6e, 0x74, 0x65, + 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, 0x41, 0x63, 0x74, 0x6f, 0x72, 0x50, 0x72, 0x6f, 0x70, + 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x41, 0x0a, 0x09, + 0x52, 0x65, 0x62, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x12, 0x34, 0x0a, 0x0a, 0x70, 0x65, 0x65, + 0x72, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, + 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x52, 0x09, 0x70, 0x65, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x22, + 0x36, 0x0a, 0x11, 0x52, 0x65, 0x62, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x43, 0x6f, 0x6d, 0x70, + 0x6c, 0x65, 0x74, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x65, 0x65, 0x72, 0x5f, 0x61, 0x64, 0x64, + 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x70, 0x65, 0x65, 0x72, + 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x42, 0xa3, 0x01, 0x0a, 0x0e, 0x63, 0x6f, 0x6d, 0x2e, + 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x42, 0x0a, 0x50, 0x65, 0x65, 0x72, + 0x73, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x48, 0x02, 0x50, 0x01, 0x5a, 0x3b, 0x67, 0x69, 0x74, 0x68, + 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x74, 0x6f, 0x63, 0x68, 0x65, 0x6d, 0x65, 0x79, 0x2f, + 0x67, 0x6f, 0x61, 0x6b, 0x74, 0x2f, 0x76, 0x33, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, + 0x6c, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x3b, 0x69, 0x6e, 0x74, + 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0xa2, 0x02, 0x03, 0x49, 0x58, 0x58, 0xaa, 0x02, 0x0a, + 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0xca, 0x02, 0x0a, 0x49, 0x6e, 0x74, + 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0xe2, 0x02, 0x16, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, + 0x61, 0x6c, 0x70, 0x62, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, + 0xea, 0x02, 0x0a, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x62, 0x06, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x33, }) var ( @@ -242,15 +245,17 @@ var file_internal_peers_proto_goTypes = []any{ (*Rebalance)(nil), // 1: internalpb.Rebalance (*RebalanceComplete)(nil), // 2: internalpb.RebalanceComplete nil, // 3: internalpb.PeerState.ActorsEntry + (*ActorProps)(nil), // 4: internalpb.ActorProps } var file_internal_peers_proto_depIdxs = []int32{ 3, // 0: internalpb.PeerState.actors:type_name -> internalpb.PeerState.ActorsEntry 0, // 1: internalpb.Rebalance.peer_state:type_name -> internalpb.PeerState - 2, // [2:2] is the sub-list for method output_type - 2, // [2:2] is the sub-list for method input_type - 2, // [2:2] is the sub-list for extension type_name - 2, // [2:2] is the sub-list for extension extendee - 0, // [0:2] is the sub-list for field type_name + 4, // 2: internalpb.PeerState.ActorsEntry.value:type_name -> internalpb.ActorProps + 3, // [3:3] is the sub-list for method output_type + 3, // [3:3] is the sub-list for method input_type + 3, // [3:3] is the sub-list for extension type_name + 3, // [3:3] is the sub-list for extension extendee + 0, // [0:3] is the sub-list for field type_name } func init() { file_internal_peers_proto_init() } @@ -258,6 +263,7 @@ func file_internal_peers_proto_init() { if File_internal_peers_proto != nil { return } + file_internal_actor_proto_init() type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ diff --git a/internal/internalpb/remoting.pb.go b/internal/internalpb/remoting.pb.go index e738f76a..75faece2 100644 --- a/internal/internalpb/remoting.pb.go +++ b/internal/internalpb/remoting.pb.go @@ -589,7 +589,9 @@ type RemoteSpawnRequest struct { // Specifies the actor name. ActorName string `protobuf:"bytes,3,opt,name=actor_name,json=actorName,proto3" json:"actor_name,omitempty"` // Specifies the actor type - ActorType string `protobuf:"bytes,4,opt,name=actor_type,json=actorType,proto3" json:"actor_type,omitempty"` + ActorType string `protobuf:"bytes,4,opt,name=actor_type,json=actorType,proto3" json:"actor_type,omitempty"` + // Specifies if the actor is a singleton + IsSingleton bool `protobuf:"varint,5,opt,name=is_singleton,json=isSingleton,proto3" json:"is_singleton,omitempty"` unknownFields protoimpl.UnknownFields sizeCache protoimpl.SizeCache } @@ -652,6 +654,13 @@ func (x *RemoteSpawnRequest) GetActorType() string { return "" } +func (x *RemoteSpawnRequest) GetIsSingleton() bool { + if x != nil { + return x.IsSingleton + } + return false +} + type RemoteSpawnResponse struct { state protoimpl.MessageState `protogen:"open.v1"` unknownFields protoimpl.UnknownFields @@ -688,6 +697,42 @@ func (*RemoteSpawnResponse) Descriptor() ([]byte, []int) { return file_internal_remoting_proto_rawDescGZIP(), []int{12} } +type SpawnOption struct { + state protoimpl.MessageState `protogen:"open.v1"` + unknownFields protoimpl.UnknownFields + sizeCache protoimpl.SizeCache +} + +func (x *SpawnOption) Reset() { + *x = SpawnOption{} + mi := &file_internal_remoting_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SpawnOption) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SpawnOption) ProtoMessage() {} + +func (x *SpawnOption) ProtoReflect() protoreflect.Message { + mi := &file_internal_remoting_proto_msgTypes[13] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SpawnOption.ProtoReflect.Descriptor instead. +func (*SpawnOption) Descriptor() ([]byte, []int) { + return file_internal_remoting_proto_rawDescGZIP(), []int{13} +} + var File_internal_remoting_proto protoreflect.FileDescriptor var file_internal_remoting_proto_rawDesc = string([]byte{ @@ -750,59 +795,62 @@ var file_internal_remoting_proto_rawDesc = string([]byte{ 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x04, 0x70, 0x6f, 0x72, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x14, 0x0a, 0x12, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x7a, 0x0a, 0x12, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, - 0x53, 0x70, 0x61, 0x77, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, - 0x68, 0x6f, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x68, 0x6f, 0x73, 0x74, - 0x12, 0x12, 0x0a, 0x04, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x04, - 0x70, 0x6f, 0x72, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x61, 0x63, 0x74, 0x6f, 0x72, 0x5f, 0x6e, 0x61, - 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x61, 0x63, 0x74, 0x6f, 0x72, 0x4e, - 0x61, 0x6d, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x61, 0x63, 0x74, 0x6f, 0x72, 0x5f, 0x74, 0x79, 0x70, - 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x61, 0x63, 0x74, 0x6f, 0x72, 0x54, 0x79, - 0x70, 0x65, 0x22, 0x15, 0x0a, 0x13, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x53, 0x70, 0x61, 0x77, - 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x32, 0xf4, 0x03, 0x0a, 0x0f, 0x52, 0x65, - 0x6d, 0x6f, 0x74, 0x69, 0x6e, 0x67, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x4c, 0x0a, - 0x09, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x41, 0x73, 0x6b, 0x12, 0x1c, 0x2e, 0x69, 0x6e, 0x74, - 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x41, 0x73, - 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, - 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x41, 0x73, 0x6b, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x28, 0x01, 0x30, 0x01, 0x12, 0x4d, 0x0a, 0x0a, 0x52, - 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x54, 0x65, 0x6c, 0x6c, 0x12, 0x1d, 0x2e, 0x69, 0x6e, 0x74, 0x65, - 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x54, 0x65, 0x6c, - 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, - 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x54, 0x65, 0x6c, 0x6c, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x28, 0x01, 0x12, 0x51, 0x0a, 0x0c, 0x52, 0x65, - 0x6d, 0x6f, 0x74, 0x65, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x12, 0x1f, 0x2e, 0x69, 0x6e, 0x74, - 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x4c, 0x6f, - 0x6f, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x69, 0x6e, - 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x4c, - 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, - 0x0d, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x53, 0x70, 0x61, 0x77, 0x6e, 0x12, 0x20, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x9d, 0x01, 0x0a, 0x12, 0x52, 0x65, 0x6d, 0x6f, 0x74, + 0x65, 0x53, 0x70, 0x61, 0x77, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, + 0x04, 0x68, 0x6f, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x68, 0x6f, 0x73, + 0x74, 0x12, 0x12, 0x0a, 0x04, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, + 0x04, 0x70, 0x6f, 0x72, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x61, 0x63, 0x74, 0x6f, 0x72, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x61, 0x63, 0x74, 0x6f, 0x72, + 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x61, 0x63, 0x74, 0x6f, 0x72, 0x5f, 0x74, 0x79, + 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x61, 0x63, 0x74, 0x6f, 0x72, 0x54, + 0x79, 0x70, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x73, 0x5f, 0x73, 0x69, 0x6e, 0x67, 0x6c, 0x65, + 0x74, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x69, 0x73, 0x53, 0x69, 0x6e, + 0x67, 0x6c, 0x65, 0x74, 0x6f, 0x6e, 0x22, 0x15, 0x0a, 0x13, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, + 0x53, 0x70, 0x61, 0x77, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x0d, 0x0a, + 0x0b, 0x53, 0x70, 0x61, 0x77, 0x6e, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x32, 0xf4, 0x03, 0x0a, + 0x0f, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x69, 0x6e, 0x67, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x12, 0x4c, 0x0a, 0x09, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x41, 0x73, 0x6b, 0x12, 0x1c, 0x2e, + 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x74, + 0x65, 0x41, 0x73, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x69, 0x6e, + 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x41, + 0x73, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x28, 0x01, 0x30, 0x01, 0x12, 0x4d, + 0x0a, 0x0a, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x54, 0x65, 0x6c, 0x6c, 0x12, 0x1d, 0x2e, 0x69, + 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, + 0x54, 0x65, 0x6c, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x69, 0x6e, + 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x54, + 0x65, 0x6c, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x28, 0x01, 0x12, 0x51, 0x0a, + 0x0c, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x12, 0x1f, 0x2e, + 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x74, + 0x65, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, - 0x74, 0x65, 0x52, 0x65, 0x53, 0x70, 0x61, 0x77, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x21, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, 0x52, 0x65, - 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x53, 0x70, 0x61, 0x77, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x4b, 0x0a, 0x0a, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x53, 0x74, 0x6f, - 0x70, 0x12, 0x1d, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, 0x52, - 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x1e, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, 0x52, 0x65, - 0x6d, 0x6f, 0x74, 0x65, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x12, 0x4e, 0x0a, 0x0b, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x53, 0x70, 0x61, 0x77, 0x6e, 0x12, - 0x1e, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, - 0x6f, 0x74, 0x65, 0x53, 0x70, 0x61, 0x77, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x1f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, 0x52, 0x65, 0x6d, - 0x6f, 0x74, 0x65, 0x53, 0x70, 0x61, 0x77, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x42, 0xa6, 0x01, 0x0a, 0x0e, 0x63, 0x6f, 0x6d, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, - 0x6c, 0x70, 0x62, 0x42, 0x0d, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x69, 0x6e, 0x67, 0x50, 0x72, 0x6f, - 0x74, 0x6f, 0x48, 0x02, 0x50, 0x01, 0x5a, 0x3b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, - 0x6f, 0x6d, 0x2f, 0x74, 0x6f, 0x63, 0x68, 0x65, 0x6d, 0x65, 0x79, 0x2f, 0x67, 0x6f, 0x61, 0x6b, - 0x74, 0x2f, 0x76, 0x33, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x69, 0x6e, - 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x3b, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, - 0x6c, 0x70, 0x62, 0xa2, 0x02, 0x03, 0x49, 0x58, 0x58, 0xaa, 0x02, 0x0a, 0x49, 0x6e, 0x74, 0x65, - 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0xca, 0x02, 0x0a, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, - 0x6c, 0x70, 0x62, 0xe2, 0x02, 0x16, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, - 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0a, 0x49, - 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x33, + 0x74, 0x65, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x54, 0x0a, 0x0d, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x53, 0x70, 0x61, 0x77, + 0x6e, 0x12, 0x20, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, 0x52, + 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x53, 0x70, 0x61, 0x77, 0x6e, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, + 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x52, 0x65, 0x53, 0x70, 0x61, 0x77, 0x6e, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4b, 0x0a, 0x0a, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, + 0x53, 0x74, 0x6f, 0x70, 0x12, 0x1d, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, + 0x62, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, + 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x53, 0x74, 0x6f, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x4e, 0x0a, 0x0b, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x53, 0x70, 0x61, + 0x77, 0x6e, 0x12, 0x1e, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, + 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x53, 0x70, 0x61, 0x77, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x2e, + 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x53, 0x70, 0x61, 0x77, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x42, 0xa6, 0x01, 0x0a, 0x0e, 0x63, 0x6f, 0x6d, 0x2e, 0x69, 0x6e, 0x74, 0x65, + 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x42, 0x0d, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x69, 0x6e, 0x67, + 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x48, 0x02, 0x50, 0x01, 0x5a, 0x3b, 0x67, 0x69, 0x74, 0x68, 0x75, + 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x74, 0x6f, 0x63, 0x68, 0x65, 0x6d, 0x65, 0x79, 0x2f, 0x67, + 0x6f, 0x61, 0x6b, 0x74, 0x2f, 0x76, 0x33, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, + 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x3b, 0x69, 0x6e, 0x74, 0x65, + 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0xa2, 0x02, 0x03, 0x49, 0x58, 0x58, 0xaa, 0x02, 0x0a, 0x49, + 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0xca, 0x02, 0x0a, 0x49, 0x6e, 0x74, 0x65, + 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0xe2, 0x02, 0x16, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, + 0x6c, 0x70, 0x62, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, + 0x02, 0x0a, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x33, }) var ( @@ -817,7 +865,7 @@ func file_internal_remoting_proto_rawDescGZIP() []byte { return file_internal_remoting_proto_rawDescData } -var file_internal_remoting_proto_msgTypes = make([]protoimpl.MessageInfo, 13) +var file_internal_remoting_proto_msgTypes = make([]protoimpl.MessageInfo, 14) var file_internal_remoting_proto_goTypes = []any{ (*RemoteAskRequest)(nil), // 0: internalpb.RemoteAskRequest (*RemoteAskResponse)(nil), // 1: internalpb.RemoteAskResponse @@ -832,19 +880,20 @@ var file_internal_remoting_proto_goTypes = []any{ (*RemoteStopResponse)(nil), // 10: internalpb.RemoteStopResponse (*RemoteSpawnRequest)(nil), // 11: internalpb.RemoteSpawnRequest (*RemoteSpawnResponse)(nil), // 12: internalpb.RemoteSpawnResponse - (*durationpb.Duration)(nil), // 13: google.protobuf.Duration - (*anypb.Any)(nil), // 14: google.protobuf.Any - (*goaktpb.Address)(nil), // 15: goaktpb.Address + (*SpawnOption)(nil), // 13: internalpb.SpawnOption + (*durationpb.Duration)(nil), // 14: google.protobuf.Duration + (*anypb.Any)(nil), // 15: google.protobuf.Any + (*goaktpb.Address)(nil), // 16: goaktpb.Address } var file_internal_remoting_proto_depIdxs = []int32{ 6, // 0: internalpb.RemoteAskRequest.remote_message:type_name -> internalpb.RemoteMessage - 13, // 1: internalpb.RemoteAskRequest.timeout:type_name -> google.protobuf.Duration - 14, // 2: internalpb.RemoteAskResponse.message:type_name -> google.protobuf.Any + 14, // 1: internalpb.RemoteAskRequest.timeout:type_name -> google.protobuf.Duration + 15, // 2: internalpb.RemoteAskResponse.message:type_name -> google.protobuf.Any 6, // 3: internalpb.RemoteTellRequest.remote_message:type_name -> internalpb.RemoteMessage - 15, // 4: internalpb.RemoteLookupResponse.address:type_name -> goaktpb.Address - 15, // 5: internalpb.RemoteMessage.sender:type_name -> goaktpb.Address - 15, // 6: internalpb.RemoteMessage.receiver:type_name -> goaktpb.Address - 14, // 7: internalpb.RemoteMessage.message:type_name -> google.protobuf.Any + 16, // 4: internalpb.RemoteLookupResponse.address:type_name -> goaktpb.Address + 16, // 5: internalpb.RemoteMessage.sender:type_name -> goaktpb.Address + 16, // 6: internalpb.RemoteMessage.receiver:type_name -> goaktpb.Address + 15, // 7: internalpb.RemoteMessage.message:type_name -> google.protobuf.Any 0, // 8: internalpb.RemotingService.RemoteAsk:input_type -> internalpb.RemoteAskRequest 2, // 9: internalpb.RemotingService.RemoteTell:input_type -> internalpb.RemoteTellRequest 4, // 10: internalpb.RemotingService.RemoteLookup:input_type -> internalpb.RemoteLookupRequest @@ -875,7 +924,7 @@ func file_internal_remoting_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: unsafe.Slice(unsafe.StringData(file_internal_remoting_proto_rawDesc), len(file_internal_remoting_proto_rawDesc)), NumEnums: 0, - NumMessages: 13, + NumMessages: 14, NumExtensions: 0, NumServices: 1, }, diff --git a/internal/types/registry.go b/internal/types/registry.go index f738f404..2f6574c8 100644 --- a/internal/types/registry.go +++ b/internal/types/registry.go @@ -27,7 +27,8 @@ package types import ( "reflect" "strings" - "sync" + + "github.com/tochemey/goakt/v3/internal/collection/syncmap" ) // Registry defines the types registry interface @@ -47,8 +48,7 @@ type Registry interface { } type registry struct { - mu *sync.RWMutex - typesMap map[string]reflect.Type + m *syncmap.Map[string, reflect.Type] } var _ Registry = (*registry)(nil) @@ -56,57 +56,45 @@ var _ Registry = (*registry)(nil) // NewRegistry creates a new types registry func NewRegistry() Registry { return ®istry{ - mu: &sync.RWMutex{}, - typesMap: make(map[string]reflect.Type), + m: syncmap.New[string, reflect.Type](), } } // Deregister removes the registered object from the registry -func (r *registry) Deregister(v any) { - r.mu.Lock() - delete(r.typesMap, TypeName(v)) - r.mu.Unlock() +func (x *registry) Deregister(v any) { + x.m.Delete(Name(v)) } // Exists return true when a given object is in the registry -func (r *registry) Exists(v any) bool { - r.mu.RLock() - defer r.mu.RUnlock() - _, ok := r.typesMap[TypeName(v)] +func (x *registry) Exists(v any) bool { + _, ok := x.m.Get(Name(v)) return ok } // TypesMap returns the list of registered at any point in time -func (r *registry) TypesMap() map[string]reflect.Type { - r.mu.Lock() - out := r.typesMap - r.mu.Unlock() +func (x *registry) TypesMap() map[string]reflect.Type { + out := make(map[string]reflect.Type) + x.m.Range(func(s string, r reflect.Type) { + out[s] = r + }) return out } // Register an object -func (r *registry) Register(v any) { +func (x *registry) Register(v any) { rtype := reflectType(v) - name := TypeName(v) - r.mu.Lock() - r.typesMap[name] = rtype - r.mu.Unlock() + name := Name(v) + x.m.Set(name, rtype) } // Type returns the type of object -func (r *registry) Type(v any) (reflect.Type, bool) { - r.mu.Lock() - defer r.mu.Unlock() - out, ok := r.typesMap[TypeName(v)] - return out, ok +func (x *registry) Type(v any) (reflect.Type, bool) { + return x.m.Get(Name(v)) } // TypeOf returns the type of object name -func (r *registry) TypeOf(name string) (reflect.Type, bool) { - r.mu.Lock() - defer r.mu.Unlock() - out, ok := r.typesMap[lowTrim(name)] - return out, ok +func (x *registry) TypeOf(name string) (reflect.Type, bool) { + return x.m.Get(lowTrim(name)) } // reflectType returns the runtime type of object @@ -121,8 +109,8 @@ func reflectType(v any) reflect.Type { return rtype } -// TypeName returns the name of a given object -func TypeName(v any) string { +// Name returns the name of a given object +func Name(v any) string { return lowTrim(reflectType(v).String()) } diff --git a/internal/types/registry_test.go b/internal/types/registry_test.go index 5da93861..ece3f7a3 100644 --- a/internal/types/registry_test.go +++ b/internal/types/registry_test.go @@ -45,6 +45,7 @@ func TestRegistry(t *testing.T) { registry := NewRegistry() // create an instance of an object obj := new(testStruct) + // register that actor registry.Register(obj) _, ok := registry.Type(obj) diff --git a/internal/util/util.go b/internal/util/util.go index 5747b21e..a817ea1f 100644 --- a/internal/util/util.go +++ b/internal/util/util.go @@ -26,16 +26,9 @@ package util import ( "time" - - "github.com/tochemey/goakt/v3/internal/types" ) // Pause pauses the running process for some time period func Pause(duration time.Duration) { - stopCh := make(chan types.Unit, 1) - timer := time.AfterFunc(duration, func() { - stopCh <- types.Unit{} - }) - <-stopCh - timer.Stop() + time.Sleep(duration) } diff --git a/protos/internal/actor.proto b/protos/internal/actor.proto index 8b247132..7ddf9f03 100644 --- a/protos/internal/actor.proto +++ b/protos/internal/actor.proto @@ -12,4 +12,17 @@ message ActorRef { goaktpb.Address actor_address = 1; // Specifies the actor type string actor_type = 2; + // Specifies if the actor is a singleton + bool is_singleton = 3; +} + +// ActorProps defines the properties of an actor +// that can be used to spawn an actor remotely. +message ActorProps { + // Specifies the actor name. + string actor_name = 1; + // Specifies the actor type + string actor_type = 2; + // Specifies if the actor is a singleton + bool is_singleton = 3; } diff --git a/protos/internal/peers.proto b/protos/internal/peers.proto index b4364da9..8dd76eb3 100644 --- a/protos/internal/peers.proto +++ b/protos/internal/peers.proto @@ -2,6 +2,8 @@ syntax = "proto3"; package internalpb; +import "internal/actor.proto"; + option go_package = "github.com/tochemey/goakt/v3/internal/internalpb;internalpb"; message PeerState { @@ -12,8 +14,8 @@ message PeerState { // Specifies the remoting host int32 peers_port = 3; // Specifies the list of actors - // actorName -> actorKind - map actors = 4; + // actorName -> ActorProps + map actors = 4; } message Rebalance { diff --git a/protos/internal/remoting.proto b/protos/internal/remoting.proto index dc2f68b7..5e6b19bc 100644 --- a/protos/internal/remoting.proto +++ b/protos/internal/remoting.proto @@ -111,6 +111,10 @@ message RemoteSpawnRequest { string actor_name = 3; // Specifies the actor type string actor_type = 4; + // Specifies if the actor is a singleton + bool is_singleton = 5; } message RemoteSpawnResponse {} + +message SpawnOption {} diff --git a/remote/config_test.go b/remote/config_test.go index 5b359f37..d1b9f570 100644 --- a/remote/config_test.go +++ b/remote/config_test.go @@ -64,7 +64,7 @@ func TestConfig(t *testing.T) { assert.EqualError(t, err, "maxFrameSize must be between 16KB and 16MB") }) t.Run("With invalid bindAddr", func(t *testing.T) { - config := NewConfig("invalid@adrr", 8080, WithMaxFrameSize(20*size.MB)) + config := NewConfig("256.256.256.256", 8080, WithMaxFrameSize(20*size.MB)) err := config.Sanitize() require.Error(t, err) })