forked from ray-project/ray
/
node_manager.cc
1386 lines (1254 loc) · 63.1 KB
/
node_manager.cc
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
#include "ray/raylet/node_manager.h"
#include "common_protocol.h"
// TODO: While removing "local_scheduler_generated.h", remove the dependency
// gen_local_scheduler_fbs from src/ray/CMakeLists.txt.
#include "local_scheduler/format/local_scheduler_generated.h"
#include "ray/raylet/format/node_manager_generated.h"
namespace {
namespace local_scheduler_protocol = ray::local_scheduler::protocol;
#define RAY_CHECK_ENUM(x, y) \
static_assert(static_cast<int>(x) == static_cast<int>(y), "protocol mismatch")
// Check consistency between client and server protocol.
RAY_CHECK_ENUM(protocol::MessageType::SubmitTask,
local_scheduler_protocol::MessageType::SubmitTask);
RAY_CHECK_ENUM(protocol::MessageType::TaskDone,
local_scheduler_protocol::MessageType::TaskDone);
RAY_CHECK_ENUM(protocol::MessageType::EventLogMessage,
local_scheduler_protocol::MessageType::EventLogMessage);
RAY_CHECK_ENUM(protocol::MessageType::RegisterClientRequest,
local_scheduler_protocol::MessageType::RegisterClientRequest);
RAY_CHECK_ENUM(protocol::MessageType::RegisterClientReply,
local_scheduler_protocol::MessageType::RegisterClientReply);
RAY_CHECK_ENUM(protocol::MessageType::DisconnectClient,
local_scheduler_protocol::MessageType::DisconnectClient);
RAY_CHECK_ENUM(protocol::MessageType::GetTask,
local_scheduler_protocol::MessageType::GetTask);
RAY_CHECK_ENUM(protocol::MessageType::ExecuteTask,
local_scheduler_protocol::MessageType::ExecuteTask);
RAY_CHECK_ENUM(protocol::MessageType::ReconstructObjects,
local_scheduler_protocol::MessageType::ReconstructObjects);
RAY_CHECK_ENUM(protocol::MessageType::NotifyUnblocked,
local_scheduler_protocol::MessageType::NotifyUnblocked);
RAY_CHECK_ENUM(protocol::MessageType::PutObject,
local_scheduler_protocol::MessageType::PutObject);
RAY_CHECK_ENUM(protocol::MessageType::GetActorFrontierRequest,
local_scheduler_protocol::MessageType::GetActorFrontierRequest);
RAY_CHECK_ENUM(protocol::MessageType::GetActorFrontierReply,
local_scheduler_protocol::MessageType::GetActorFrontierReply);
RAY_CHECK_ENUM(protocol::MessageType::SetActorFrontier,
local_scheduler_protocol::MessageType::SetActorFrontier);
/// A helper function to determine whether a given actor task has already been executed
/// according to the given actor registry. Returns true if the task is a duplicate.
bool CheckDuplicateActorTask(
const std::unordered_map<ActorID, ray::raylet::ActorRegistration> &actor_registry,
const ray::raylet::TaskSpecification &spec) {
auto actor_entry = actor_registry.find(spec.ActorId());
RAY_CHECK(actor_entry != actor_registry.end());
const auto &frontier = actor_entry->second.GetFrontier();
int64_t expected_task_counter = 0;
auto frontier_entry = frontier.find(spec.ActorHandleId());
if (frontier_entry != frontier.end()) {
expected_task_counter = frontier_entry->second.task_counter;
}
if (spec.ActorCounter() < expected_task_counter) {
// The assigned task counter is less than expected. The actor has already
// executed past this task, so do not assign the task again.
RAY_LOG(WARNING) << "A task was resubmitted, so we are ignoring it. This "
<< "should only happen during reconstruction.";
return true;
}
RAY_CHECK(spec.ActorCounter() == expected_task_counter)
<< "Expected actor counter: " << expected_task_counter
<< ", got: " << spec.ActorCounter();
return false;
};
} // namespace
namespace ray {
namespace raylet {
NodeManager::NodeManager(boost::asio::io_service &io_service,
const NodeManagerConfig &config, ObjectManager &object_manager,
std::shared_ptr<gcs::AsyncGcsClient> gcs_client)
: io_service_(io_service),
object_manager_(object_manager),
gcs_client_(gcs_client),
heartbeat_timer_(io_service),
heartbeat_period_(std::chrono::milliseconds(config.heartbeat_period_ms)),
local_resources_(config.resource_config),
local_available_resources_(config.resource_config),
worker_pool_(config.num_initial_workers, config.num_workers_per_process,
static_cast<int>(config.resource_config.GetNumCpus()),
config.worker_command),
local_queues_(SchedulingQueue()),
scheduling_policy_(local_queues_),
reconstruction_policy_(
io_service_,
[this](const TaskID &task_id) { HandleTaskReconstruction(task_id); },
RayConfig::instance().initial_reconstruction_timeout_milliseconds(),
gcs_client_->client_table().GetLocalClientId(), gcs_client->task_lease_table(),
std::make_shared<ObjectDirectory>(gcs_client),
gcs_client_->task_reconstruction_log()),
task_dependency_manager_(
object_manager, reconstruction_policy_, io_service,
gcs_client_->client_table().GetLocalClientId(),
RayConfig::instance().initial_reconstruction_timeout_milliseconds(),
gcs_client->task_lease_table()),
lineage_cache_(gcs_client_->client_table().GetLocalClientId(),
gcs_client->raylet_task_table(), gcs_client->raylet_task_table(),
config.max_lineage_size),
remote_clients_(),
remote_server_connections_(),
actor_registry_() {
RAY_CHECK(heartbeat_period_.count() > 0);
// Initialize the resource map with own cluster resource configuration.
ClientID local_client_id = gcs_client_->client_table().GetLocalClientId();
cluster_resource_map_.emplace(local_client_id,
SchedulingResources(config.resource_config));
RAY_CHECK_OK(object_manager_.SubscribeObjAdded([this](const ObjectInfoT &object_info) {
ObjectID object_id = ObjectID::from_binary(object_info.object_id);
HandleObjectLocal(object_id);
}));
RAY_CHECK_OK(object_manager_.SubscribeObjDeleted(
[this](const ObjectID &object_id) { HandleObjectMissing(object_id); }));
ARROW_CHECK_OK(store_client_.Connect(config.store_socket_name.c_str(), "", 0));
}
ray::Status NodeManager::RegisterGcs() {
object_manager_.RegisterGcs();
// Subscribe to task entry commits in the GCS. These notifications are
// forwarded to the lineage cache, which requests notifications about tasks
// that were executed remotely.
const auto task_committed_callback = [this](gcs::AsyncGcsClient *client,
const TaskID &task_id,
const ray::protocol::TaskT &task_data) {
lineage_cache_.HandleEntryCommitted(task_id);
};
RAY_RETURN_NOT_OK(gcs_client_->raylet_task_table().Subscribe(
JobID::nil(), gcs_client_->client_table().GetLocalClientId(),
task_committed_callback, nullptr, nullptr));
const auto task_lease_notification_callback = [this](gcs::AsyncGcsClient *client,
const TaskID &task_id,
const TaskLeaseDataT &task_lease) {
const ClientID node_manager_id = ClientID::from_binary(task_lease.node_manager_id);
if (gcs_client_->client_table().IsRemoved(node_manager_id)) {
// The node manager that added the task lease is already removed. The
// lease is considered inactive.
reconstruction_policy_.HandleTaskLeaseNotification(task_id, 0);
} else {
// NOTE(swang): The task_lease.timeout is an overestimate of the lease's
// expiration period since the entry may have been in the GCS for some
// time already. For a more accurate estimate, the age of the entry in
// the GCS should be subtracted from task_lease.timeout.
reconstruction_policy_.HandleTaskLeaseNotification(task_id, task_lease.timeout);
}
};
const auto task_lease_empty_callback = [this](gcs::AsyncGcsClient *client,
const TaskID &task_id) {
reconstruction_policy_.HandleTaskLeaseNotification(task_id, 0);
};
RAY_RETURN_NOT_OK(gcs_client_->task_lease_table().Subscribe(
JobID::nil(), gcs_client_->client_table().GetLocalClientId(),
task_lease_notification_callback, task_lease_empty_callback, nullptr));
// Register a callback for actor creation notifications.
auto actor_creation_callback = [this](
gcs::AsyncGcsClient *client, const ActorID &actor_id,
const std::vector<ActorTableDataT> &data) { HandleActorCreation(actor_id, data); };
RAY_RETURN_NOT_OK(gcs_client_->actor_table().Subscribe(
UniqueID::nil(), UniqueID::nil(), actor_creation_callback, nullptr));
// Register a callback on the client table for new clients.
auto node_manager_client_added = [this](gcs::AsyncGcsClient *client, const UniqueID &id,
const ClientTableDataT &data) {
ClientAdded(data);
};
gcs_client_->client_table().RegisterClientAddedCallback(node_manager_client_added);
// Register a callback on the client table for removed clients.
auto node_manager_client_removed = [this](
gcs::AsyncGcsClient *client, const UniqueID &id, const ClientTableDataT &data) {
ClientRemoved(data);
};
gcs_client_->client_table().RegisterClientRemovedCallback(node_manager_client_removed);
// Subscribe to node manager heartbeats.
const auto heartbeat_added = [this](gcs::AsyncGcsClient *client, const ClientID &id,
const HeartbeatTableDataT &heartbeat_data) {
HeartbeatAdded(client, id, heartbeat_data);
};
RAY_RETURN_NOT_OK(gcs_client_->heartbeat_table().Subscribe(
UniqueID::nil(), UniqueID::nil(), heartbeat_added, nullptr,
[](gcs::AsyncGcsClient *client) {
RAY_LOG(DEBUG) << "heartbeat table subscription done callback called.";
}));
// Subscribe to driver table updates.
const auto driver_table_handler = [this](
gcs::AsyncGcsClient *client, const ClientID &client_id,
const std::vector<DriverTableDataT> &driver_data) {
HandleDriverTableUpdate(client_id, driver_data);
};
RAY_RETURN_NOT_OK(gcs_client_->driver_table().Subscribe(JobID::nil(), UniqueID::nil(),
driver_table_handler, nullptr));
// Start sending heartbeats to the GCS.
last_heartbeat_at_ms_ = current_time_ms();
Heartbeat();
return ray::Status::OK();
}
void NodeManager::HandleDriverTableUpdate(
const ClientID &id, const std::vector<DriverTableDataT> &driver_data) {
for (const auto &entry : driver_data) {
RAY_LOG(DEBUG) << "HandleDriverTableUpdate " << UniqueID::from_binary(entry.driver_id)
<< " " << entry.is_dead;
if (entry.is_dead) {
// TODO: Implement cleanup on driver death. For reference,
// see handle_driver_removed_callback in local_scheduler.cc
}
}
}
void NodeManager::Heartbeat() {
uint64_t now_ms = current_time_ms();
uint64_t interval = now_ms - last_heartbeat_at_ms_;
if (interval > RayConfig::instance().num_heartbeats_warning() *
RayConfig::instance().heartbeat_timeout_milliseconds()) {
RAY_LOG(WARNING) << "Last heartbeat was sent " << interval << " ms ago ";
}
last_heartbeat_at_ms_ = now_ms;
RAY_LOG(DEBUG) << "[Heartbeat] sending heartbeat.";
auto &heartbeat_table = gcs_client_->heartbeat_table();
auto heartbeat_data = std::make_shared<HeartbeatTableDataT>();
auto client_id = gcs_client_->client_table().GetLocalClientId();
const SchedulingResources &local_resources = cluster_resource_map_[client_id];
heartbeat_data->client_id = client_id.hex();
// TODO(atumanov): modify the heartbeat table protocol to use the ResourceSet directly.
// TODO(atumanov): implement a ResourceSet const_iterator.
for (const auto &resource_pair :
local_resources.GetAvailableResources().GetResourceMap()) {
heartbeat_data->resources_available_label.push_back(resource_pair.first);
heartbeat_data->resources_available_capacity.push_back(resource_pair.second);
}
for (const auto &resource_pair : local_resources.GetTotalResources().GetResourceMap()) {
heartbeat_data->resources_total_label.push_back(resource_pair.first);
heartbeat_data->resources_total_capacity.push_back(resource_pair.second);
}
ray::Status status = heartbeat_table.Add(
UniqueID::nil(), gcs_client_->client_table().GetLocalClientId(), heartbeat_data,
[](ray::gcs::AsyncGcsClient *client, const ClientID &id,
const HeartbeatTableDataT &data) {
RAY_LOG(DEBUG) << "[HEARTBEAT] heartbeat sent callback";
});
if (!status.ok()) {
RAY_LOG(INFO) << "heartbeat failed: string " << status.ToString() << status.message();
RAY_LOG(INFO) << "is redis error: " << status.IsRedisError();
}
RAY_CHECK_OK(status);
// Reset the timer.
heartbeat_timer_.expires_from_now(heartbeat_period_);
heartbeat_timer_.async_wait([this](const boost::system::error_code &error) {
RAY_CHECK(!error);
Heartbeat();
});
}
void NodeManager::ClientAdded(const ClientTableDataT &client_data) {
ClientID client_id = ClientID::from_binary(client_data.client_id);
RAY_LOG(DEBUG) << "[ClientAdded] received callback from client id " << client_id;
if (client_id == gcs_client_->client_table().GetLocalClientId()) {
// We got a notification for ourselves, so we are connected to the GCS now.
// Save this NodeManager's resource information in the cluster resource map.
cluster_resource_map_[client_id] = local_resources_;
return;
}
// TODO(atumanov): make remote client lookup O(1)
if (std::find(remote_clients_.begin(), remote_clients_.end(), client_id) ==
remote_clients_.end()) {
RAY_LOG(DEBUG) << "a new client: " << client_id;
remote_clients_.push_back(client_id);
} else {
// NodeManager connection to this client was already established.
RAY_LOG(DEBUG) << "received a new client connection that already exists: "
<< client_id;
return;
}
ResourceSet resources_total(client_data.resources_total_label,
client_data.resources_total_capacity);
this->cluster_resource_map_.emplace(client_id, SchedulingResources(resources_total));
// Establish a new NodeManager connection to this GCS client.
auto client_info = gcs_client_->client_table().GetClient(client_id);
RAY_LOG(DEBUG) << "[ClientAdded] CONNECTING TO: "
<< " " << client_info.node_manager_address << " "
<< client_info.node_manager_port;
boost::asio::ip::tcp::socket socket(io_service_);
RAY_CHECK_OK(TcpConnect(socket, client_info.node_manager_address,
client_info.node_manager_port));
auto server_conn = TcpServerConnection(std::move(socket));
remote_server_connections_.emplace(client_id, std::move(server_conn));
}
void NodeManager::ClientRemoved(const ClientTableDataT &client_data) {
// TODO(swang): If we receive a notification for our own death, clean up and
// exit immediately.
const ClientID client_id = ClientID::from_binary(client_data.client_id);
RAY_LOG(DEBUG) << "[ClientRemoved] received callback from client id " << client_id;
RAY_CHECK(client_id != gcs_client_->client_table().GetLocalClientId())
<< "Exiting because this node manager has mistakenly been marked dead by the "
<< "monitor.";
// Below, when we remove client_id from all of these data structures, we could
// check that it is actually removed, or log a warning otherwise, but that may
// not be necessary.
// Remove the client from the list of remote clients.
std::remove(remote_clients_.begin(), remote_clients_.end(), client_id);
// Remove the client from the resource map.
cluster_resource_map_.erase(client_id);
// Remove the remote server connection.
remote_server_connections_.erase(client_id);
}
void NodeManager::HeartbeatAdded(gcs::AsyncGcsClient *client, const ClientID &client_id,
const HeartbeatTableDataT &heartbeat_data) {
RAY_LOG(DEBUG) << "[HeartbeatAdded]: received heartbeat from client id " << client_id;
if (client_id == gcs_client_->client_table().GetLocalClientId()) {
// Skip heartbeats from self.
return;
}
// Locate the client id in remote client table and update available resources based on
// the received heartbeat information.
auto it = this->cluster_resource_map_.find(client_id);
if (it == cluster_resource_map_.end()) {
// Haven't received the client registration for this client yet, skip this heartbeat.
RAY_LOG(INFO) << "[HeartbeatAdded]: received heartbeat from unknown client id "
<< client_id;
return;
}
SchedulingResources &resources = it->second;
ResourceSet heartbeat_resource_available(heartbeat_data.resources_available_label,
heartbeat_data.resources_available_capacity);
resources.SetAvailableResources(
ResourceSet(heartbeat_data.resources_available_label,
heartbeat_data.resources_available_capacity));
RAY_CHECK(this->cluster_resource_map_[client_id].GetAvailableResources() ==
heartbeat_resource_available);
}
void NodeManager::HandleActorCreation(const ActorID &actor_id,
const std::vector<ActorTableDataT> &data) {
RAY_LOG(DEBUG) << "Actor creation notification received: " << actor_id;
// TODO(swang): In presence of failures, data may have size > 1, since the
// actor will have been created multiple times. In that case, we should
// only consider the last entry as valid. All previous entries should have
// a dead node_manager_id.
RAY_CHECK(data.size() == 1);
// Register the new actor.
ActorRegistration actor_registration(data.back());
ClientID received_node_manager_id = actor_registration.GetNodeManagerId();
// Extend the frontier to include the actor creation task. NOTE(swang): The
// creator of the actor is always assigned nil as the actor handle ID.
actor_registration.ExtendFrontier(ActorHandleID::nil(),
actor_registration.GetActorCreationDependency());
auto inserted = actor_registry_.emplace(actor_id, std::move(actor_registration));
if (!inserted.second) {
// If we weren't able to insert the actor's location, check that the
// existing entry is the same as the new one.
// TODO(swang): This is not true in the case of failures.
RAY_CHECK(received_node_manager_id == inserted.first->second.GetNodeManagerId())
<< "Actor scheduled on " << inserted.first->second.GetNodeManagerId()
<< ", but received notification for " << received_node_manager_id;
} else {
// The actor's location is now known. Dequeue any methods that were
// submitted before the actor's location was known.
// (See design_docs/task_states.rst for the state transition diagram.)
const auto &methods = local_queues_.GetMethodsWaitingForActorCreation();
std::unordered_set<TaskID> created_actor_method_ids;
for (const auto &method : methods) {
if (method.GetTaskSpecification().ActorId() == actor_id) {
created_actor_method_ids.insert(method.GetTaskSpecification().TaskId());
}
}
// Resubmit the methods that were submitted before the actor's location was
// known.
auto created_actor_methods = local_queues_.RemoveTasks(created_actor_method_ids);
for (const auto &method : created_actor_methods) {
lineage_cache_.RemoveWaitingTask(method.GetTaskSpecification().TaskId());
// The task's uncommitted lineage was already added to the local lineage
// cache upon the initial submission, so it's okay to resubmit it with an
// empty lineage this time.
SubmitTask(method, Lineage());
}
}
}
void NodeManager::GetActorTasksFromList(const ActorID &actor_id,
const std::list<Task> &tasks,
std::unordered_set<TaskID> &tasks_to_remove) {
for (auto const &task : tasks) {
auto const &spec = task.GetTaskSpecification();
if (actor_id == spec.ActorId()) {
tasks_to_remove.insert(spec.TaskId());
}
}
}
void NodeManager::CleanUpTasksForDeadActor(const ActorID &actor_id) {
// TODO(rkn): The code below should be cleaned up when we improve the
// SchedulingQueue API.
std::unordered_set<TaskID> tasks_to_remove;
// (See design_docs/task_states.rst for the state transition diagram.)
GetActorTasksFromList(actor_id, local_queues_.GetMethodsWaitingForActorCreation(),
tasks_to_remove);
GetActorTasksFromList(actor_id, local_queues_.GetWaitingTasks(), tasks_to_remove);
GetActorTasksFromList(actor_id, local_queues_.GetPlaceableTasks(), tasks_to_remove);
GetActorTasksFromList(actor_id, local_queues_.GetReadyTasks(), tasks_to_remove);
GetActorTasksFromList(actor_id, local_queues_.GetRunningTasks(), tasks_to_remove);
GetActorTasksFromList(actor_id, local_queues_.GetBlockedTasks(), tasks_to_remove);
auto removed_tasks = local_queues_.RemoveTasks(tasks_to_remove);
for (auto const &task : removed_tasks) {
const TaskSpecification &spec = task.GetTaskSpecification();
TreatTaskAsFailed(spec);
task_dependency_manager_.TaskCanceled(spec.TaskId());
}
}
void NodeManager::ProcessNewClient(LocalClientConnection &client) {
// The new client is a worker, so begin listening for messages.
client.ProcessMessages();
}
void NodeManager::DispatchTasks() {
// Work with a copy of scheduled tasks.
// (See design_docs/task_states.rst for the state transition diagram.)
auto scheduled_tasks = local_queues_.GetReadyTasks();
// Return if there are no tasks to schedule.
if (scheduled_tasks.empty()) {
return;
}
for (const auto &task : scheduled_tasks) {
const auto &task_resources = task.GetTaskSpecification().GetRequiredResources();
if (!local_available_resources_.Contains(task_resources)) {
// Not enough local resources for this task right now, skip this task.
// TODO(rkn): We should always skip node managers that have 0 CPUs.
continue;
}
// We have enough resources for this task. Assign task.
// TODO(atumanov): perform the task state/queue transition inside AssignTask.
// (See design_docs/task_states.rst for the state transition diagram.)
auto dispatched_task = local_queues_.RemoveTask(task.GetTaskSpecification().TaskId());
AssignTask(dispatched_task);
}
}
void NodeManager::ProcessClientMessage(
const std::shared_ptr<LocalClientConnection> &client, int64_t message_type,
const uint8_t *message_data) {
RAY_LOG(DEBUG) << "Message of type " << message_type;
switch (static_cast<protocol::MessageType>(message_type)) {
case protocol::MessageType::RegisterClientRequest: {
auto message = flatbuffers::GetRoot<protocol::RegisterClientRequest>(message_data);
client->SetClientID(from_flatbuf(*message->client_id()));
auto worker = std::make_shared<Worker>(message->worker_pid(), client);
if (message->is_worker()) {
// Register the new worker.
worker_pool_.RegisterWorker(std::move(worker));
} else {
// Register the new driver.
JobID job_id = from_flatbuf(*message->driver_id());
worker->AssignTaskId(job_id);
worker_pool_.RegisterDriver(std::move(worker));
local_queues_.AddDriverTaskId(job_id);
}
} break;
case protocol::MessageType::GetTask: {
std::shared_ptr<Worker> worker = worker_pool_.GetRegisteredWorker(client);
RAY_CHECK(worker);
// If the worker was assigned a task, mark it as finished.
if (!worker->GetAssignedTaskId().is_nil()) {
FinishAssignedTask(*worker);
}
// Return the worker to the idle pool.
worker_pool_.PushWorker(std::move(worker));
// Call task dispatch to assign work to the new worker.
DispatchTasks();
} break;
case protocol::MessageType::DisconnectClient: {
// Remove the dead worker from the pool and stop listening for messages.
const std::shared_ptr<Worker> worker = worker_pool_.GetRegisteredWorker(client);
if (worker) {
// The client is a worker. Handle the case where the worker is killed
// while executing a task. Clean up the assigned task's resources, push
// an error to the driver.
// (See design_docs/task_states.rst for the state transition diagram.)
const TaskID &task_id = worker->GetAssignedTaskId();
if (!task_id.is_nil()) {
auto const &running_tasks = local_queues_.GetRunningTasks();
// TODO(rkn): This is too heavyweight just to get the task's driver ID.
auto const it = std::find_if(
running_tasks.begin(), running_tasks.end(), [task_id](const Task &task) {
return task.GetTaskSpecification().TaskId() == task_id;
});
RAY_CHECK(running_tasks.size() != 0);
RAY_CHECK(it != running_tasks.end());
const TaskSpecification &spec = it->GetTaskSpecification();
const JobID job_id = spec.DriverId();
// TODO(rkn): Define this constant somewhere else.
std::string type = "worker_died";
std::ostringstream error_message;
error_message << "A worker died or was killed while executing task " << task_id
<< ".";
RAY_CHECK_OK(gcs_client_->error_table().PushErrorToDriver(
job_id, type, error_message.str(), current_time_ms()));
// Handle the task failure in order to raise an exception in the
// application.
TreatTaskAsFailed(spec);
task_dependency_manager_.TaskCanceled(spec.TaskId());
local_queues_.RemoveTask(spec.TaskId());
}
worker_pool_.DisconnectWorker(worker);
// If the worker was an actor, add it to the list of dead actors.
const ActorID actor_id = worker->GetActorId();
if (!actor_id.is_nil()) {
// TODO(rkn): Consider broadcasting a message to all of the other
// node managers so that they can mark the actor as dead.
RAY_LOG(DEBUG) << "The actor with ID " << actor_id << " died.";
auto actor_entry = actor_registry_.find(actor_id);
RAY_CHECK(actor_entry != actor_registry_.end());
actor_entry->second.MarkDead();
// For dead actors, if there are remaining tasks for this actor, we
// should handle them.
CleanUpTasksForDeadActor(actor_id);
}
const ClientID &client_id = gcs_client_->client_table().GetLocalClientId();
// Return the resources that were being used by this worker.
auto const &task_resources = worker->GetTaskResourceIds();
local_available_resources_.Release(task_resources);
cluster_resource_map_[client_id].Release(task_resources.ToResourceSet());
worker->ResetTaskResourceIds();
auto const &lifetime_resources = worker->GetLifetimeResourceIds();
local_available_resources_.Release(lifetime_resources);
cluster_resource_map_[client_id].Release(lifetime_resources.ToResourceSet());
worker->ResetLifetimeResourceIds();
// Since some resources may have been released, we can try to dispatch more tasks.
DispatchTasks();
} else {
// The client is a driver.
RAY_CHECK_OK(gcs_client_->driver_table().AppendDriverData(client->GetClientID(),
/*is_dead=*/true));
const std::shared_ptr<Worker> driver = worker_pool_.GetRegisteredDriver(client);
RAY_CHECK(driver);
auto driver_id = driver->GetAssignedTaskId();
RAY_CHECK(!driver_id.is_nil());
local_queues_.RemoveDriverTaskId(driver_id);
worker_pool_.DisconnectDriver(driver);
}
return;
} break;
case protocol::MessageType::SubmitTask: {
// Read the task submitted by the client.
auto message = flatbuffers::GetRoot<protocol::SubmitTaskRequest>(message_data);
TaskExecutionSpecification task_execution_spec(
from_flatbuf(*message->execution_dependencies()));
TaskSpecification task_spec(*message->task_spec());
Task task(task_execution_spec, task_spec);
// Submit the task to the local scheduler. Since the task was submitted
// locally, there is no uncommitted lineage.
SubmitTask(task, Lineage());
} break;
case protocol::MessageType::ReconstructObjects: {
auto message = flatbuffers::GetRoot<protocol::ReconstructObjects>(message_data);
std::vector<ObjectID> required_object_ids;
for (size_t i = 0; i < message->object_ids()->size(); ++i) {
ObjectID object_id = from_flatbuf(*message->object_ids()->Get(i));
if (!task_dependency_manager_.CheckObjectLocal(object_id)) {
if (message->fetch_only()) {
// If only a fetch is required, then do not subscribe to the
// dependencies to the task dependency manager.
RAY_CHECK_OK(object_manager_.Pull(object_id));
} else {
// If reconstruction is also required, then add any missing objects
// to the list to subscribe to in the task dependency manager. These
// objects will be pulled from remote node managers and reconstructed
// if necessary.
required_object_ids.push_back(object_id);
}
}
}
if (!required_object_ids.empty()) {
std::shared_ptr<Worker> worker = worker_pool_.GetRegisteredWorker(client);
if (worker) {
// The client is a worker. Mark the worker as blocked. This
// temporarily releases any resources that the worker holds while it is
// blocked.
HandleWorkerBlocked(worker);
} else {
// The client is a driver. Drivers do not hold resources, so we simply
// mark the driver as blocked.
worker = worker_pool_.GetRegisteredDriver(client);
RAY_CHECK(worker);
worker->MarkBlocked();
}
const TaskID current_task_id = worker->GetAssignedTaskId();
RAY_CHECK(!current_task_id.is_nil());
// Subscribe to the objects required by the ray.get. These objects will
// be fetched and/or reconstructed as necessary, until the objects become
// local or are unsubscribed.
task_dependency_manager_.SubscribeDependencies(current_task_id,
required_object_ids);
}
} break;
case protocol::MessageType::NotifyUnblocked: {
std::shared_ptr<Worker> worker = worker_pool_.GetRegisteredWorker(client);
// Re-acquire the CPU resources for the task that was assigned to the
// unblocked worker.
// TODO(swang): Because the object dependencies are tracked in the task
// dependency manager, we could actually remove this message entirely and
// instead unblock the worker once all the objects become available.
bool was_blocked;
if (worker) {
was_blocked = worker->IsBlocked();
// Mark the worker as unblocked. This returns the temporarily released
// resources to the worker.
HandleWorkerUnblocked(worker);
} else {
// The client is a driver. Drivers do not hold resources, so we simply
// mark the driver as unblocked.
worker = worker_pool_.GetRegisteredDriver(client);
RAY_CHECK(worker);
was_blocked = worker->IsBlocked();
worker->MarkUnblocked();
}
// Unsubscribe to the objects. Any fetch or reconstruction operations to
// make the objects local are canceled.
if (was_blocked) {
const TaskID current_task_id = worker->GetAssignedTaskId();
RAY_CHECK(!current_task_id.is_nil());
task_dependency_manager_.UnsubscribeDependencies(current_task_id);
}
} break;
case protocol::MessageType::WaitRequest: {
// Read the data.
auto message = flatbuffers::GetRoot<protocol::WaitRequest>(message_data);
std::vector<ObjectID> object_ids = from_flatbuf(*message->object_ids());
int64_t wait_ms = message->timeout();
uint64_t num_required_objects = static_cast<uint64_t>(message->num_ready_objects());
bool wait_local = message->wait_local();
ray::Status status = object_manager_.Wait(
object_ids, wait_ms, num_required_objects, wait_local,
[client](std::vector<ObjectID> found, std::vector<ObjectID> remaining) {
// Write the data.
flatbuffers::FlatBufferBuilder fbb;
flatbuffers::Offset<protocol::WaitReply> wait_reply = protocol::CreateWaitReply(
fbb, to_flatbuf(fbb, found), to_flatbuf(fbb, remaining));
fbb.Finish(wait_reply);
RAY_CHECK_OK(
client->WriteMessage(static_cast<int64_t>(protocol::MessageType::WaitReply),
fbb.GetSize(), fbb.GetBufferPointer()));
});
RAY_CHECK_OK(status);
} break;
case protocol::MessageType::PushErrorRequest: {
auto message = flatbuffers::GetRoot<protocol::PushErrorRequest>(message_data);
JobID job_id = from_flatbuf(*message->job_id());
auto const &type = string_from_flatbuf(*message->type());
auto const &error_message = string_from_flatbuf(*message->error_message());
double timestamp = message->timestamp();
RAY_CHECK_OK(gcs_client_->error_table().PushErrorToDriver(job_id, type, error_message,
timestamp));
} break;
case protocol::MessageType::PushProfileEventsRequest: {
auto message = flatbuffers::GetRoot<ProfileTableData>(message_data);
RAY_CHECK_OK(gcs_client_->profile_table().AddProfileEventBatch(*message));
} break;
default:
RAY_LOG(FATAL) << "Received unexpected message type " << message_type;
}
// Listen for more messages.
client->ProcessMessages();
}
void NodeManager::ProcessNewNodeManager(TcpClientConnection &node_manager_client) {
node_manager_client.ProcessMessages();
}
void NodeManager::ProcessNodeManagerMessage(TcpClientConnection &node_manager_client,
int64_t message_type,
const uint8_t *message_data) {
switch (static_cast<protocol::MessageType>(message_type)) {
case protocol::MessageType::ForwardTaskRequest: {
auto message = flatbuffers::GetRoot<protocol::ForwardTaskRequest>(message_data);
TaskID task_id = from_flatbuf(*message->task_id());
Lineage uncommitted_lineage(*message);
const Task &task = uncommitted_lineage.GetEntry(task_id)->TaskData();
RAY_LOG(DEBUG) << "got task " << task.GetTaskSpecification().TaskId()
<< " spillback=" << task.GetTaskExecutionSpecReadonly().NumForwards();
SubmitTask(task, uncommitted_lineage, /* forwarded = */ true);
} break;
case protocol::MessageType::DisconnectClient: {
// TODO(rkn): We need to do some cleanup here.
RAY_LOG(DEBUG) << "Received disconnect message from remote node manager. "
<< "We need to do some cleanup here.";
} break;
default:
RAY_LOG(FATAL) << "Received unexpected message type " << message_type;
}
node_manager_client.ProcessMessages();
}
void NodeManager::ScheduleTasks() {
auto policy_decision = scheduling_policy_.Schedule(
cluster_resource_map_, gcs_client_->client_table().GetLocalClientId(),
remote_clients_);
#ifndef NDEBUG
RAY_LOG(DEBUG) << "[NM ScheduleTasks] policy decision:";
for (const auto &pair : policy_decision) {
TaskID task_id = pair.first;
ClientID client_id = pair.second;
RAY_LOG(DEBUG) << task_id << " --> " << client_id;
}
#endif
// Extract decision for this local scheduler.
std::unordered_set<TaskID> local_task_ids;
// Iterate over (taskid, clientid) pairs, extract tasks assigned to the local node.
for (const auto &task_schedule : policy_decision) {
const TaskID task_id = task_schedule.first;
const ClientID client_id = task_schedule.second;
if (client_id == gcs_client_->client_table().GetLocalClientId()) {
local_task_ids.insert(task_id);
} else {
// TODO(atumanov): need a better interface for task exit on forward.
// (See design_docs/task_states.rst for the state transition diagram.)
const auto task = local_queues_.RemoveTask(task_id);
// Attempt to forward the task. If this fails to forward the task,
// the task will be resubmit locally.
ForwardTaskOrResubmit(task, client_id);
}
}
// Transition locally placed tasks to waiting or ready for dispatch.
if (local_task_ids.size() > 0) {
std::vector<Task> tasks = local_queues_.RemoveTasks(local_task_ids);
for (const auto &t : tasks) {
EnqueuePlaceableTask(t);
}
}
// All remaining placeable tasks should be registered with the task dependency
// manager. TaskDependencyManager::TaskPending() is assumed to be idempotent.
// TODO(atumanov): evaluate performance implications of registering all new tasks on
// submission vs. registering remaining queued placeable tasks here.
for (const auto &task : local_queues_.GetPlaceableTasks()) {
task_dependency_manager_.TaskPending(task);
}
}
void NodeManager::TreatTaskAsFailed(const TaskSpecification &spec) {
RAY_LOG(DEBUG) << "Treating task " << spec.TaskId() << " as failed.";
// Loop over the return IDs (except the dummy ID) and store a fake object in
// the object store.
int64_t num_returns = spec.NumReturns();
if (spec.IsActorTask()) {
// TODO(rkn): We subtract 1 to avoid the dummy ID. However, this leaks
// information about the TaskSpecification implementation.
num_returns -= 1;
}
for (int64_t i = 0; i < num_returns; i++) {
const ObjectID object_id = spec.ReturnId(i);
std::shared_ptr<Buffer> data;
// TODO(ekl): this writes an invalid arrow object, which is sufficient to
// signal that the worker failed, but it would be nice to return more
// detailed failure metadata in the future.
arrow::Status status =
store_client_.Create(object_id.to_plasma_id(), 1, NULL, 0, &data);
if (!status.IsPlasmaObjectExists()) {
// TODO(rkn): We probably don't want this checks. E.g., if the object
// store is full, we don't want to kill the raylet.
ARROW_CHECK_OK(status);
ARROW_CHECK_OK(store_client_.Seal(object_id.to_plasma_id()));
}
}
}
void NodeManager::SubmitTask(const Task &task, const Lineage &uncommitted_lineage,
bool forwarded) {
if (local_queues_.HasTask(task.GetTaskSpecification().TaskId())) {
RAY_LOG(WARNING) << "Submitted task " << task.GetTaskSpecification().TaskId()
<< " is already queued and will not be reconstructed. This is most "
"likely due to spurious reconstruction.";
return;
}
// Add the task and its uncommitted lineage to the lineage cache.
lineage_cache_.AddWaitingTask(task, uncommitted_lineage);
const TaskSpecification &spec = task.GetTaskSpecification();
if (spec.IsActorTask()) {
// Check whether we know the location of the actor.
const auto actor_entry = actor_registry_.find(spec.ActorId());
if (actor_entry != actor_registry_.end()) {
// We have a known location for the actor.
auto node_manager_id = actor_entry->second.GetNodeManagerId();
if (node_manager_id == gcs_client_->client_table().GetLocalClientId()) {
// The actor is local. Check if the actor is still alive.
if (!actor_entry->second.IsAlive()) {
// Handle the fact that this actor is dead.
TreatTaskAsFailed(spec);
} else {
// Queue the task for local execution, bypassing placement.
EnqueuePlaceableTask(task);
}
} else {
// The actor is remote. Forward the task to the node manager that owns
// the actor.
if (gcs_client_->client_table().IsRemoved(node_manager_id)) {
// The remote node manager is dead, so handle the fact that this actor
// is also dead.
TreatTaskAsFailed(spec);
} else {
// Attempt to forward the task. If this fails to forward the task,
// the task will be resubmit locally.
ForwardTaskOrResubmit(task, node_manager_id);
}
}
} else {
// We do not have a registered location for the object, so either the
// actor has not yet been created or we missed the notification for the
// actor creation because this node joined the cluster after the actor
// was already created. Look up the actor's registered location in case
// we missed the creation notification.
// NOTE(swang): This codepath needs to be tested in a cluster setting.
auto lookup_callback = [this](gcs::AsyncGcsClient *client, const ActorID &actor_id,
const std::vector<ActorTableDataT> &data) {
if (!data.empty()) {
// The actor has been created.
HandleActorCreation(actor_id, data);
} else {
// The actor has not yet been created.
// TODO(swang): Set a timer for reconstructing the actor creation
// task.
}
};
RAY_CHECK_OK(gcs_client_->actor_table().Lookup(JobID::nil(), spec.ActorId(),
lookup_callback));
// Keep the task queued until we discover the actor's location.
// (See design_docs/task_states.rst for the state transition diagram.)
local_queues_.QueueMethodsWaitingForActorCreation({task});
// Mark the task as pending. It will be canceled once we discover the
// actor's location and either execute the task ourselves or forward it
// to another node.
task_dependency_manager_.TaskPending(task);
}
} else {
// This is a non-actor task. Queue the task for a placement decision or for dispatch
// if the task was forwarded.
if (forwarded) {
// Check for local dependencies and enqueue as waiting or ready for dispatch.
EnqueuePlaceableTask(task);
} else {
// (See design_docs/task_states.rst for the state transition diagram.)
local_queues_.QueuePlaceableTasks({task});
ScheduleTasks();
}
}
}
void NodeManager::HandleWorkerBlocked(std::shared_ptr<Worker> worker) {
RAY_CHECK(worker);
if (worker->IsBlocked()) {
return;
}
// If the worker isn't already blocked, then release any CPU resources that
// it acquired for its assigned task while it is blocked. The resources will
// be acquired again once the worker is unblocked.
RAY_CHECK(!worker->GetAssignedTaskId().is_nil());
// (See design_docs/task_states.rst for the state transition diagram.)
const auto task = local_queues_.RemoveTask(worker->GetAssignedTaskId());
// Get the CPU resources required by the running task.
const auto required_resources = task.GetTaskSpecification().GetRequiredResources();
double required_cpus = required_resources.GetNumCpus();
const std::unordered_map<std::string, double> cpu_resources = {
{kCPU_ResourceLabel, required_cpus}};
// Release the CPU resources.
auto const cpu_resource_ids = worker->ReleaseTaskCpuResources();
local_available_resources_.Release(cpu_resource_ids);
RAY_CHECK(cluster_resource_map_[gcs_client_->client_table().GetLocalClientId()].Release(
ResourceSet(cpu_resources)));
// Mark the task as blocked.
local_queues_.QueueBlockedTasks({task});
worker->MarkBlocked();
// Try to dispatch more tasks since the blocked worker released some
// resources.
DispatchTasks();
}
void NodeManager::HandleWorkerUnblocked(std::shared_ptr<Worker> worker) {
RAY_CHECK(worker);
if (!worker->IsBlocked()) {
return;
}
// (See design_docs/task_states.rst for the state transition diagram.)
const auto task = local_queues_.RemoveTask(worker->GetAssignedTaskId());
// Get the CPU resources required by the running task.
const auto required_resources = task.GetTaskSpecification().GetRequiredResources();
double required_cpus = required_resources.GetNumCpus();
const ResourceSet cpu_resources(
std::unordered_map<std::string, double>({{kCPU_ResourceLabel, required_cpus}}));
// Check if we can reacquire the CPU resources.
bool oversubscribed = !local_available_resources_.Contains(cpu_resources);
if (!oversubscribed) {
// Reacquire the CPU resources for the worker. Note that care needs to be
// taken if the user is using the specific CPU IDs since the IDs that we
// reacquire here may be different from the ones that the task started with.
auto const resource_ids = local_available_resources_.Acquire(cpu_resources);
worker->AcquireTaskCpuResources(resource_ids);
RAY_CHECK(
cluster_resource_map_[gcs_client_->client_table().GetLocalClientId()].Acquire(
cpu_resources));
} else {
// In this case, we simply don't reacquire the CPU resources for the worker.
// The worker can keep running and when the task finishes, it will simply
// not have any CPU resources to release.
RAY_LOG(WARNING)
<< "Resources oversubscribed: "
<< cluster_resource_map_[gcs_client_->client_table().GetLocalClientId()]
.GetAvailableResources()
.ToString();
}
// Mark the task as running again.
// (See design_docs/task_states.rst for the state transition diagram.)
local_queues_.QueueRunningTasks({task});
worker->MarkUnblocked();
}
void NodeManager::EnqueuePlaceableTask(const Task &task) {
// Mark the task as pending. Once the task has finished execution, or once it
// has been forwarded to another node, the task must be marked as canceled in
// the TaskDependencyManager.
task_dependency_manager_.TaskPending(task);
// TODO(atumanov): add task lookup hashmap and change EnqueuePlaceableTask to take
// a vector of TaskIDs. Trigger MoveTask internally.
// Subscribe to the task's dependencies.
bool args_ready = task_dependency_manager_.SubscribeDependencies(
task.GetTaskSpecification().TaskId(), task.GetDependencies());
// Enqueue the task. If all dependencies are available, then the task is queued
// in the READY state, else the WAITING state.
// (See design_docs/task_states.rst for the state transition diagram.)
if (args_ready) {
local_queues_.QueueReadyTasks({task});
// Try to dispatch the newly ready task.
DispatchTasks();
} else {