@@ -310,18 +310,19 @@ impl LocalStreamManager {
310
310
Ok ( ( ) )
311
311
}
312
312
313
- pub async fn drop_actor ( & self , actors : & [ ActorId ] ) -> StreamResult < ( ) > {
313
+ /// Drop the resources of the given actors.
314
+ pub async fn drop_actors ( & self , actors : & [ ActorId ] ) -> StreamResult < ( ) > {
314
315
let mut core = self . core . lock ( ) . await ;
315
- for id in actors {
316
- core. drop_actor ( * id) ;
316
+ for & id in actors {
317
+ core. drop_actor ( id) ;
317
318
}
318
319
tracing:: debug!( actors = ?actors, "drop actors" ) ;
319
320
Ok ( ( ) )
320
321
}
321
322
322
- /// Force stop all actors on this worker.
323
+ /// Force stop all actors on this worker, and then drop their resources .
323
324
pub async fn stop_all_actors ( & self ) -> StreamResult < ( ) > {
324
- self . core . lock ( ) . await . drop_all_actors ( ) . await ;
325
+ self . core . lock ( ) . await . stop_all_actors ( ) . await ;
325
326
// Clear shared buffer in storage to release memory
326
327
self . clear_storage_buffer ( ) . await ;
327
328
self . clear_all_senders_and_collect_rx ( ) ;
@@ -557,7 +558,7 @@ impl LocalStreamManagerCore {
557
558
558
559
// If there're multiple stateful executors in this actor, we will wrap it into a subtask.
559
560
let executor = if has_stateful && is_stateful {
560
- let ( subtask, executor) = subtask:: wrap ( executor) ;
561
+ let ( subtask, executor) = subtask:: wrap ( executor, actor_context . id ) ;
561
562
subtasks. push ( subtask) ;
562
563
executor. boxed ( )
563
564
} else {
@@ -781,14 +782,16 @@ impl LocalStreamManagerCore {
781
782
. inspect ( |handle| handle. abort ( ) ) ;
782
783
self . context . actor_infos . write ( ) . remove ( & actor_id) ;
783
784
self . actors . remove ( & actor_id) ;
784
- // Task should have already stopped when this method is invoked.
785
- self . handles
786
- . remove ( & actor_id)
787
- . inspect ( |handle| handle. abort ( ) ) ;
785
+
786
+ // Task should have already stopped when this method is invoked. There might be some
787
+ // clean-up work left (like dropping in-memory data structures), but we don't have to wait
788
+ // for them to finish, in order to make this request non-blocking.
789
+ self . handles . remove ( & actor_id) ;
788
790
}
789
791
790
- /// `drop_all_actors` is invoked by meta node via RPC for recovery purpose.
791
- async fn drop_all_actors ( & mut self ) {
792
+ /// `stop_all_actors` is invoked by meta node via RPC for recovery purpose. Different from the
793
+ /// `drop_actor`, the execution of the actors will be aborted.
794
+ async fn stop_all_actors ( & mut self ) {
792
795
for ( actor_id, handle) in & self . handles {
793
796
tracing:: debug!( "force stopping actor {}" , actor_id) ;
794
797
handle. abort ( ) ;
0 commit comments