@@ -171,16 +171,9 @@ void TaskReceiver::HandleTask(rpc::PushTaskRequest request,
171171 if (canceled_task_spec.IsActorTask ()) {
172172 // If task cancelation is due to worker shutdown, propagate that information
173173 // to the submitter.
174- bool is_worker_exiting = false ;
175- {
176- absl::MutexLock lock (&stop_mu_);
177- is_worker_exiting = stopping_;
178- if (stopping_) {
179- reply->set_worker_exiting (true );
180- reply->set_was_cancelled_before_running (true );
181- }
182- }
183- if (is_worker_exiting) {
174+ if (stopping_) {
175+ reply->set_worker_exiting (true );
176+ reply->set_was_cancelled_before_running (true );
184177 canceled_send_reply_callback (Status::OK (), nullptr , nullptr );
185178 } else {
186179 canceled_send_reply_callback (status, nullptr , nullptr );
@@ -191,82 +184,79 @@ void TaskReceiver::HandleTask(rpc::PushTaskRequest request,
191184 }
192185 };
193186
194- {
195- absl::MutexLock lock (&stop_mu_);
196- task_spec = TaskSpecification (std::move (*request.mutable_task_spec ()));
197- if (stopping_) {
198- reply->set_was_cancelled_before_running (true );
199- if (task_spec.IsActorTask ()) {
200- reply->set_worker_exiting (true );
201- }
202- send_reply_callback (Status::OK (), nullptr , nullptr );
203- return ;
187+ task_spec = TaskSpecification (std::move (*request.mutable_task_spec ()));
188+ if (stopping_) {
189+ reply->set_was_cancelled_before_running (true );
190+ if (task_spec.IsActorTask ()) {
191+ reply->set_worker_exiting (true );
204192 }
193+ send_reply_callback (Status::OK (), nullptr , nullptr );
194+ return ;
195+ }
205196
206- if (task_spec.IsActorCreationTask ()) {
207- SetupActor (task_spec.IsAsyncioActor (),
208- task_spec.MaxActorConcurrency (),
209- task_spec.AllowOutOfOrderExecution ());
210- }
197+ if (task_spec.IsActorCreationTask ()) {
198+ SetupActor (task_spec.IsAsyncioActor (),
199+ task_spec.MaxActorConcurrency (),
200+ task_spec.AllowOutOfOrderExecution ());
201+ }
211202
212- if (!task_spec.IsActorTask ()) {
213- resource_ids = ResourceMappingType{};
214- for (const auto &mapping : request.resource_mapping ()) {
215- std::vector<std::pair<int64_t , double >> rids;
216- rids.reserve (mapping.resource_ids ().size ());
217- for (const auto &ids : mapping.resource_ids ()) {
218- rids.emplace_back (ids.index (), ids.quantity ());
219- }
220- (*resource_ids)[mapping.name ()] = std::move (rids);
203+ if (!task_spec.IsActorTask ()) {
204+ resource_ids = ResourceMappingType{};
205+ for (const auto &mapping : request.resource_mapping ()) {
206+ std::vector<std::pair<int64_t , double >> rids;
207+ rids.reserve (mapping.resource_ids ().size ());
208+ for (const auto &ids : mapping.resource_ids ()) {
209+ rids.emplace_back (ids.index (), ids.quantity ());
221210 }
211+ (*resource_ids)[mapping.name ()] = std::move (rids);
222212 }
213+ }
223214
224- if (task_spec.IsActorTask ()) {
225- auto it = actor_scheduling_queues_.find (task_spec.CallerWorkerId ());
226- if (it == actor_scheduling_queues_.end ()) {
227- it = actor_scheduling_queues_
228- .emplace (
229- task_spec.CallerWorkerId (),
230- allow_out_of_order_execution_
231- ? std::unique_ptr<SchedulingQueue>(
232- std::make_unique<OutOfOrderActorSchedulingQueue>(
233- task_execution_service_,
234- waiter_,
235- task_event_buffer_,
236- pool_manager_,
237- fiber_state_manager_,
238- is_asyncio_,
239- fiber_max_concurrency_,
240- concurrency_groups_))
241- : std::unique_ptr<
242- SchedulingQueue>(std::make_unique<ActorSchedulingQueue>(
243- task_execution_service_,
244- waiter_,
245- task_event_buffer_,
246- pool_manager_,
247- RayConfig::instance ()
248- .actor_scheduling_queue_max_reorder_wait_seconds ())))
249- .first ;
250- }
251-
252- auto accept_callback = make_accept_callback ();
253- it->second ->Add (request.sequence_number (),
254- request.client_processed_up_to (),
255- std::move (accept_callback),
256- std::move (cancel_callback),
257- std::move (send_reply_callback),
258- std::move (task_spec));
259- } else {
260- RAY_LOG (DEBUG) << " Adding task " << task_spec.TaskId ()
261- << " to normal scheduling task queue." ;
262- auto accept_callback = make_accept_callback ();
263- normal_scheduling_queue_->Add (request.sequence_number (),
264- request.client_processed_up_to (),
265- std::move (accept_callback),
266- std::move (cancel_callback),
267- std::move (send_reply_callback),
268- std::move (task_spec));
215+ if (task_spec.IsActorTask ()) {
216+ auto it = actor_scheduling_queues_.find (task_spec.CallerWorkerId ());
217+ if (it == actor_scheduling_queues_.end ()) {
218+ it = actor_scheduling_queues_
219+ .emplace (
220+ task_spec.CallerWorkerId (),
221+ allow_out_of_order_execution_
222+ ? std::unique_ptr<SchedulingQueue>(
223+ std::make_unique<OutOfOrderActorSchedulingQueue>(
224+ task_execution_service_,
225+ waiter_,
226+ task_event_buffer_,
227+ pool_manager_,
228+ fiber_state_manager_,
229+ is_asyncio_,
230+ fiber_max_concurrency_,
231+ concurrency_groups_))
232+ : std::unique_ptr<SchedulingQueue>(
233+ std::make_unique<ActorSchedulingQueue>(
234+ task_execution_service_,
235+ waiter_,
236+ task_event_buffer_,
237+ pool_manager_,
238+ RayConfig::instance ()
239+ .actor_scheduling_queue_max_reorder_wait_seconds ())))
240+ .first ;
269241 }
242+
243+ auto accept_callback = make_accept_callback ();
244+ it->second ->Add (request.sequence_number (),
245+ request.client_processed_up_to (),
246+ std::move (accept_callback),
247+ std::move (cancel_callback),
248+ std::move (send_reply_callback),
249+ std::move (task_spec));
250+ } else {
251+ RAY_LOG (DEBUG) << " Adding task " << task_spec.TaskId ()
252+ << " to normal scheduling task queue." ;
253+ auto accept_callback = make_accept_callback ();
254+ normal_scheduling_queue_->Add (request.sequence_number (),
255+ request.client_processed_up_to (),
256+ std::move (accept_callback),
257+ std::move (cancel_callback),
258+ std::move (send_reply_callback),
259+ std::move (task_spec));
270260 }
271261}
272262
@@ -315,12 +305,8 @@ void TaskReceiver::SetupActor(bool is_asyncio,
315305}
316306
317307void TaskReceiver::Stop () {
318- {
319- absl::MutexLock lock (&stop_mu_);
320- if (stopping_) {
321- return ;
322- }
323- stopping_ = true ;
308+ if (stopping_.exchange (true )) {
309+ return ;
324310 }
325311 for (const auto &[_, scheduling_queue] : actor_scheduling_queues_) {
326312 scheduling_queue->Stop ();
0 commit comments