/
node_manager.cc
3070 lines (2831 loc) · 139 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 <cctype>
#include <fstream>
#include <memory>
#include "ray/common/status.h"
#include "ray/common/common_protocol.h"
#include "ray/common/id.h"
#include "ray/raylet/format/node_manager_generated.h"
#include "ray/stats/stats.h"
#include "ray/util/sample.h"
namespace {
#define RAY_CHECK_ENUM(x, y) \
static_assert(static_cast<int>(x) == static_cast<int>(y), "protocol mismatch")
/// A helper function to return the expected actor counter for a given actor
/// and actor handle, according to the given actor registry. If a task's
/// counter is less than the returned value, then the task is a duplicate. If
/// the task's counter is equal to the returned value, then the task should be
/// the next to run.
int64_t GetExpectedTaskCounter(
const std::unordered_map<ray::ActorID, ray::raylet::ActorRegistration>
&actor_registry,
const ray::ActorID &actor_id, const ray::TaskID &actor_caller_id) {
auto actor_entry = actor_registry.find(actor_id);
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(actor_caller_id);
if (frontier_entry != frontier.end()) {
expected_task_counter = frontier_entry->second.task_counter;
}
return expected_task_counter;
};
struct ActorStats {
int live_actors = 0;
int dead_actors = 0;
int reconstructing_actors = 0;
int max_num_handles = 0;
};
/// A helper function to return the statistical data of actors in this node manager.
ActorStats GetActorStatisticalData(
std::unordered_map<ray::ActorID, ray::raylet::ActorRegistration> actor_registry) {
ActorStats item;
for (auto &pair : actor_registry) {
if (pair.second.GetState() == ray::rpc::ActorTableData::ALIVE) {
item.live_actors += 1;
} else if (pair.second.GetState() == ray::rpc::ActorTableData::RECONSTRUCTING) {
item.reconstructing_actors += 1;
} else {
item.dead_actors += 1;
}
if (pair.second.NumHandles() > item.max_num_handles) {
item.max_num_handles = pair.second.NumHandles();
}
}
return item;
}
} // namespace
namespace ray {
namespace raylet {
NodeManager::NodeManager(boost::asio::io_service &io_service,
const NodeManagerConfig &config, ObjectManager &object_manager,
std::shared_ptr<gcs::RedisGcsClient> gcs_client,
std::shared_ptr<ObjectDirectoryInterface> object_directory)
: client_id_(gcs_client->client_table().GetLocalClientId()),
io_service_(io_service),
object_manager_(object_manager),
gcs_client_(std::move(gcs_client)),
object_directory_(std::move(object_directory)),
heartbeat_timer_(io_service),
heartbeat_period_(std::chrono::milliseconds(config.heartbeat_period_ms)),
debug_dump_period_(config.debug_dump_period_ms),
fair_queueing_enabled_(config.fair_queueing_enabled),
temp_dir_(config.temp_dir),
object_manager_profile_timer_(io_service),
initial_config_(config),
local_available_resources_(config.resource_config),
worker_pool_(config.num_initial_workers, config.maximum_startup_concurrency,
gcs_client_, config.worker_commands),
scheduling_policy_(local_queues_),
reconstruction_policy_(
io_service_,
[this](const TaskID &task_id, const ObjectID &required_object_id) {
HandleTaskReconstruction(task_id, required_object_id);
},
RayConfig::instance().initial_reconstruction_timeout_milliseconds(),
gcs_client_->client_table().GetLocalClientId(), gcs_client_->task_lease_table(),
object_directory_, 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),
actor_registry_(),
node_manager_server_("NodeManager", config.node_manager_port),
node_manager_service_(io_service, *this),
client_call_manager_(io_service),
new_scheduler_enabled_(RayConfig::instance().new_scheduler_enabled()) {
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 object_manager::protocol::ObjectInfoT &object_info) {
ObjectID object_id = ObjectID::FromPlasmaIdBinary(object_info.object_id);
HandleObjectLocal(object_id);
}));
RAY_CHECK_OK(object_manager_.SubscribeObjDeleted(
[this](const ObjectID &object_id) { HandleObjectMissing(object_id); }));
if (new_scheduler_enabled_) {
SchedulingResources &local_resources = cluster_resource_map_[local_client_id];
new_resource_scheduler_ =
std::shared_ptr<ClusterResourceScheduler>(new ClusterResourceScheduler(
client_id_.Binary(), local_resources.GetTotalResources().GetResourceMap()));
}
RAY_ARROW_CHECK_OK(store_client_.Connect(config.store_socket_name.c_str()));
// Run the node manger rpc server.
node_manager_server_.RegisterService(node_manager_service_);
node_manager_server_.Run();
}
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::RedisGcsClient *client,
const TaskID &task_id,
const TaskTableData &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::RedisGcsClient *client,
const TaskID &task_id,
const TaskLeaseData &task_lease) {
const ClientID node_manager_id = ClientID::FromBinary(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::RedisGcsClient *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 to handle actor notifications.
auto actor_notification_callback = [this](const ActorID &actor_id,
const ActorTableData &data) {
HandleActorStateTransition(actor_id, ActorRegistration(data));
};
RAY_RETURN_NOT_OK(
gcs_client_->Actors().AsyncSubscribeAll(actor_notification_callback, nullptr));
// Register a callback on the client table for new clients.
auto node_manager_client_added = [this](gcs::RedisGcsClient *client, const UniqueID &id,
const GcsNodeInfo &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::RedisGcsClient *client,
const UniqueID &id, const GcsNodeInfo &data) {
ClientRemoved(data);
};
gcs_client_->client_table().RegisterClientRemovedCallback(node_manager_client_removed);
// Subscribe to resource changes.
const auto &resources_changed =
[this](
gcs::RedisGcsClient *client, const ClientID &id,
const gcs::GcsChangeMode change_mode,
const std::unordered_map<std::string, std::shared_ptr<gcs::ResourceTableData>>
&data) {
if (change_mode == gcs::GcsChangeMode::APPEND_OR_ADD) {
ResourceSet resource_set;
for (auto &entry : data) {
resource_set.AddOrUpdateResource(entry.first,
entry.second->resource_capacity());
}
ResourceCreateUpdated(id, resource_set);
}
if (change_mode == gcs::GcsChangeMode::REMOVE) {
std::vector<std::string> resource_names;
for (auto &entry : data) {
resource_names.push_back(entry.first);
}
ResourceDeleted(id, resource_names);
}
};
RAY_RETURN_NOT_OK(
gcs_client_->resource_table().Subscribe(JobID::Nil(), ClientID::Nil(),
/*subscribe_callback=*/resources_changed,
/*done_callback=*/nullptr));
// Subscribe to heartbeat batches from the monitor.
const auto &heartbeat_batch_added =
[this](gcs::RedisGcsClient *client, const ClientID &id,
const HeartbeatBatchTableData &heartbeat_batch) {
HeartbeatBatchAdded(heartbeat_batch);
};
RAY_RETURN_NOT_OK(gcs_client_->heartbeat_batch_table().Subscribe(
JobID::Nil(), ClientID::Nil(), heartbeat_batch_added,
/*subscribe_callback=*/nullptr,
/*done_callback=*/nullptr));
// Subscribe to driver table updates.
const auto job_table_handler = [this](gcs::RedisGcsClient *client, const JobID &job_id,
const std::vector<JobTableData> &job_data) {
HandleJobTableUpdate(job_id, job_data);
};
RAY_RETURN_NOT_OK(gcs_client_->job_table().Subscribe(JobID::Nil(), ClientID::Nil(),
job_table_handler, nullptr));
// Start sending heartbeats to the GCS.
last_heartbeat_at_ms_ = current_time_ms();
last_debug_dump_at_ms_ = current_time_ms();
Heartbeat();
// Start the timer that gets object manager profiling information and sends it
// to the GCS.
GetObjectManagerProfileInfo();
return ray::Status::OK();
}
void NodeManager::KillWorker(std::shared_ptr<Worker> worker) {
// If we're just cleaning up a single worker, allow it some time to clean
// up its state before force killing. The client socket will be closed
// and the worker struct will be freed after the timeout.
kill(worker->Pid(), SIGTERM);
auto retry_timer = std::make_shared<boost::asio::deadline_timer>(io_service_);
auto retry_duration = boost::posix_time::milliseconds(
RayConfig::instance().kill_worker_timeout_milliseconds());
retry_timer->expires_from_now(retry_duration);
retry_timer->async_wait([retry_timer, worker](const boost::system::error_code &error) {
RAY_LOG(DEBUG) << "Send SIGKILL to worker, pid=" << worker->Pid();
// Force kill worker. TODO(rkn): Is there some small danger that the worker
// has already died and the PID has been reassigned to a different process?
kill(worker->Pid(), SIGKILL);
});
}
void NodeManager::HandleJobTableUpdate(const JobID &id,
const std::vector<JobTableData> &job_data) {
for (const auto &entry : job_data) {
RAY_LOG(DEBUG) << "HandleJobTableUpdate " << JobID::FromBinary(entry.job_id()) << " "
<< entry.is_dead();
if (entry.is_dead()) {
auto job_id = JobID::FromBinary(entry.job_id());
auto workers = worker_pool_.GetWorkersRunningTasksForJob(job_id);
// Kill all the workers. The actual cleanup for these workers is done
// later when we receive the DisconnectClient message from them.
for (const auto &worker : workers) {
if (!worker->IsDetachedActor()) {
// Clean up any open ray.wait calls that the worker made.
task_dependency_manager_.UnsubscribeWaitDependencies(worker->WorkerId());
// Mark the worker as dead so further messages from it are ignored
// (except DisconnectClient).
worker->MarkDead();
// Then kill the worker process.
KillWorker(worker);
}
}
// Remove all tasks for this job from the scheduling queues, mark
// the results for these tasks as not required, cancel any attempts
// at reconstruction. Note that at this time the workers are likely
// alive because of the delay in killing workers.
auto tasks_to_remove = local_queues_.GetTaskIdsForJob(job_id);
task_dependency_manager_.RemoveTasksAndRelatedObjects(tasks_to_remove);
// NOTE(swang): SchedulingQueue::RemoveTasks modifies its argument so we must
// call it last.
local_queues_.RemoveTasks(tasks_to_remove);
}
}
}
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().raylet_heartbeat_timeout_milliseconds()) {
RAY_LOG(WARNING) << "Last heartbeat was sent " << interval << " ms ago ";
}
last_heartbeat_at_ms_ = now_ms;
auto &heartbeat_table = gcs_client_->heartbeat_table();
auto heartbeat_data = std::make_shared<HeartbeatTableData>();
const auto &my_client_id = gcs_client_->client_table().GetLocalClientId();
SchedulingResources &local_resources = cluster_resource_map_[my_client_id];
heartbeat_data->set_client_id(my_client_id.Binary());
// 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->add_resources_available_label(resource_pair.first);
heartbeat_data->add_resources_available_capacity(resource_pair.second);
}
for (const auto &resource_pair : local_resources.GetTotalResources().GetResourceMap()) {
heartbeat_data->add_resources_total_label(resource_pair.first);
heartbeat_data->add_resources_total_capacity(resource_pair.second);
}
local_resources.SetLoadResources(local_queues_.GetResourceLoad());
for (const auto &resource_pair : local_resources.GetLoadResources().GetResourceMap()) {
heartbeat_data->add_resource_load_label(resource_pair.first);
heartbeat_data->add_resource_load_capacity(resource_pair.second);
}
size_t max_size = RayConfig::instance().raylet_max_active_object_ids();
std::unordered_set<ObjectID> active_object_ids = worker_pool_.GetActiveObjectIDs();
if (active_object_ids.size() <= max_size) {
for (const auto &object_id : active_object_ids) {
heartbeat_data->add_active_object_id(object_id.Binary());
}
} else {
// If there are more than the configured maximum number of object IDs to send per
// heartbeat, sample from them randomly.
// TODO(edoakes): we might want to improve the sampling technique here, for example
// preferring object IDs with the earliest last-refreshed timestamp.
std::vector<ObjectID> downsampled;
random_sample(active_object_ids.begin(), active_object_ids.end(), max_size,
&downsampled);
for (const auto &object_id : downsampled) {
heartbeat_data->add_active_object_id(object_id.Binary());
}
}
ray::Status status = heartbeat_table.Add(
JobID::Nil(), gcs_client_->client_table().GetLocalClientId(), heartbeat_data,
/*success_callback=*/nullptr);
RAY_CHECK_OK_PREPEND(status, "Heartbeat failed");
if (debug_dump_period_ > 0 &&
static_cast<int64_t>(now_ms - last_debug_dump_at_ms_) > debug_dump_period_) {
DumpDebugState();
RecordMetrics();
WarnResourceDeadlock();
last_debug_dump_at_ms_ = now_ms;
}
// 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();
});
}
// TODO(edoakes): this function is problematic because it both sends warnings spuriously
// under normal conditions and sometimes doesn't send a warning under actual deadlock
// conditions. The current logic is to push a warning when: all running tasks are
// blocked, there is at least one ready task, and a warning hasn't been pushed in
// debug_dump_period_ milliseconds.
// See https://github.com/ray-project/ray/issues/5790 for details.
void NodeManager::WarnResourceDeadlock() {
// Check if any progress is being made on this raylet.
for (const auto &task : local_queues_.GetTasks(TaskState::RUNNING)) {
// Ignore blocked tasks.
if (local_queues_.GetBlockedTaskIds().count(task.GetTaskSpecification().TaskId())) {
continue;
}
// Progress is being made, don't warn.
resource_deadlock_warned_ = false;
return;
}
// suppress duplicates warning messages
if (resource_deadlock_warned_) {
return;
}
// The node is full of actors and no progress has been made for some time.
// If there are any pending tasks, build a warning.
std::ostringstream error_message;
ray::Task exemplar;
bool should_warn = false;
int pending_actor_creations = 0;
int pending_tasks = 0;
// See if any tasks are blocked trying to acquire resources.
for (const auto &task : local_queues_.GetTasks(TaskState::READY)) {
const TaskSpecification &spec = task.GetTaskSpecification();
if (spec.IsActorCreationTask()) {
pending_actor_creations += 1;
} else {
pending_tasks += 1;
}
if (!should_warn) {
exemplar = task;
should_warn = true;
}
}
// Push an warning to the driver that a task is blocked trying to acquire resources.
if (should_warn) {
const auto &my_client_id = gcs_client_->client_table().GetLocalClientId();
SchedulingResources &local_resources = cluster_resource_map_[my_client_id];
error_message
<< "The actor or task with ID " << exemplar.GetTaskSpecification().TaskId()
<< " is pending and cannot currently be scheduled. It requires "
<< exemplar.GetTaskSpecification().GetRequiredResources().ToString()
<< " for execution and "
<< exemplar.GetTaskSpecification().GetRequiredPlacementResources().ToString()
<< " for placement, but this node only has remaining "
<< local_resources.GetAvailableResources().ToString() << ". In total there are "
<< pending_tasks << " pending tasks and " << pending_actor_creations
<< " pending actors on this node. "
<< "This is likely due to all cluster resources being claimed by actors. "
<< "To resolve the issue, consider creating fewer actors or increase the "
<< "resources available to this Ray cluster. You can ignore this message "
<< "if this Ray cluster is expected to auto-scale.";
RAY_CHECK_OK(gcs_client_->error_table().PushErrorToDriver(
exemplar.GetTaskSpecification().JobId(), "resource_deadlock", error_message.str(),
current_time_ms()));
resource_deadlock_warned_ = true;
}
}
void NodeManager::GetObjectManagerProfileInfo() {
int64_t start_time_ms = current_time_ms();
auto profile_info = object_manager_.GetAndResetProfilingInfo();
if (profile_info.profile_events_size() > 0) {
RAY_CHECK_OK(gcs_client_->profile_table().AddProfileEventBatch(profile_info));
}
// Reset the timer.
object_manager_profile_timer_.expires_from_now(heartbeat_period_);
object_manager_profile_timer_.async_wait(
[this](const boost::system::error_code &error) {
RAY_CHECK(!error);
GetObjectManagerProfileInfo();
});
int64_t interval = current_time_ms() - start_time_ms;
if (interval > RayConfig::instance().handler_warning_timeout_ms()) {
RAY_LOG(WARNING) << "GetObjectManagerProfileInfo handler took " << interval << " ms.";
}
}
void NodeManager::ClientAdded(const GcsNodeInfo &node_info) {
const ClientID client_id = ClientID::FromBinary(node_info.node_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] = initial_config_.resource_config;
return;
}
auto entry = remote_node_manager_clients_.find(client_id);
if (entry != remote_node_manager_clients_.end()) {
RAY_LOG(DEBUG) << "Received notification of a new client that already exists: "
<< client_id;
return;
}
// Initialize a rpc client to the new node manager.
std::unique_ptr<rpc::NodeManagerClient> client(
new rpc::NodeManagerClient(node_info.node_manager_address(),
node_info.node_manager_port(), client_call_manager_));
remote_node_manager_clients_.emplace(client_id, std::move(client));
// Fetch resource info for the remote client and update cluster resource map.
RAY_CHECK_OK(gcs_client_->resource_table().Lookup(
JobID::Nil(), client_id,
[this](gcs::RedisGcsClient *client, const ClientID &client_id,
const std::unordered_map<std::string,
std::shared_ptr<gcs::ResourceTableData>> &pairs) {
ResourceSet resource_set;
for (auto &resource_entry : pairs) {
resource_set.AddOrUpdateResource(resource_entry.first,
resource_entry.second->resource_capacity());
}
ResourceCreateUpdated(client_id, resource_set);
}));
}
void NodeManager::ClientRemoved(const GcsNodeInfo &node_info) {
// TODO(swang): If we receive a notification for our own death, clean up and
// exit immediately.
const ClientID client_id = ClientID::FromBinary(node_info.node_id());
RAY_LOG(DEBUG) << "[ClientRemoved] Received callback from client id " << client_id;
if (!gcs_client_->client_table().IsDisconnected()) {
// We could receive a notification for our own death when we disconnect from client
// table after receiving a 'SIGTERM' signal, in that case we disconnect from gcs
// client table and then do some cleanup in the disconnect callback, and it's possible
// that we receive the notification in between, for more details refer to the SIGTERM
// handler in main.cc. In this case check for intentional disconnection and rule it
// out.
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 resource map.
cluster_resource_map_.erase(client_id);
// Remove the node manager client.
const auto client_entry = remote_node_manager_clients_.find(client_id);
if (client_entry != remote_node_manager_clients_.end()) {
remote_node_manager_clients_.erase(client_entry);
} else {
RAY_LOG(WARNING) << "Received ClientRemoved callback for an unknown client "
<< client_id << ".";
}
// For any live actors that were on the dead node, broadcast a notification
// about the actor's death
// TODO(swang): This could be very slow if there are many actors.
for (const auto &actor_entry : actor_registry_) {
if (actor_entry.second.GetNodeManagerId() == client_id &&
actor_entry.second.GetState() == ActorTableData::ALIVE) {
RAY_LOG(INFO) << "Actor " << actor_entry.first
<< " is disconnected, because its node " << client_id
<< " is removed from cluster. It may be reconstructed.";
HandleDisconnectedActor(actor_entry.first, /*was_local=*/false,
/*intentional_disconnect=*/false);
}
}
// Notify the object directory that the client has been removed so that it
// can remove it from any cached locations.
object_directory_->HandleClientRemoved(client_id);
// Flush all uncommitted tasks from the local lineage cache. This is to
// guarantee that all tasks get flushed eventually, in case one of the tasks
// in our local cache was supposed to be flushed by the node that died.
lineage_cache_.FlushAllUncommittedTasks();
}
void NodeManager::ResourceCreateUpdated(const ClientID &client_id,
const ResourceSet &createUpdatedResources) {
const ClientID &local_client_id = gcs_client_->client_table().GetLocalClientId();
RAY_LOG(DEBUG) << "[ResourceCreateUpdated] received callback from client id "
<< client_id << " with created or updated resources: "
<< createUpdatedResources.ToString() << ". Updating resource map.";
SchedulingResources &cluster_schedres = cluster_resource_map_[client_id];
// Update local_available_resources_ and SchedulingResources
for (const auto &resource_pair : createUpdatedResources.GetResourceMap()) {
const std::string &resource_label = resource_pair.first;
const double &new_resource_capacity = resource_pair.second;
cluster_schedres.UpdateResourceCapacity(resource_label, new_resource_capacity);
if (client_id == local_client_id) {
local_available_resources_.AddOrUpdateResource(resource_label,
new_resource_capacity);
}
if (new_scheduler_enabled_) {
new_resource_scheduler_->UpdateResourceCapacity(client_id.Binary(), resource_label,
new_resource_capacity);
}
}
RAY_LOG(DEBUG) << "[ResourceCreateUpdated] Updated cluster_resource_map.";
if (client_id == local_client_id) {
// The resource update is on the local node, check if we can reschedule tasks.
TryLocalInfeasibleTaskScheduling();
}
return;
}
void NodeManager::ResourceDeleted(const ClientID &client_id,
const std::vector<std::string> &resource_names) {
const ClientID &local_client_id = gcs_client_->client_table().GetLocalClientId();
if (RAY_LOG_ENABLED(DEBUG)) {
std::ostringstream oss;
for (auto &resource_name : resource_names) {
oss << resource_name << ", ";
}
RAY_LOG(DEBUG) << "[ResourceDeleted] received callback from client id " << client_id
<< " with deleted resources: " << oss.str()
<< ". Updating resource map.";
}
SchedulingResources &cluster_schedres = cluster_resource_map_[client_id];
// Update local_available_resources_ and SchedulingResources
for (const auto &resource_label : resource_names) {
cluster_schedres.DeleteResource(resource_label);
if (client_id == local_client_id) {
local_available_resources_.DeleteResource(resource_label);
}
if (new_scheduler_enabled_) {
new_resource_scheduler_->DeleteResource(client_id.Binary(), resource_label);
}
}
RAY_LOG(DEBUG) << "[ResourceDeleted] Updated cluster_resource_map.";
return;
}
void NodeManager::TryLocalInfeasibleTaskScheduling() {
RAY_LOG(DEBUG) << "[LocalResourceUpdateRescheduler] The resource update is on the "
"local node, check if we can reschedule tasks";
const ClientID &local_client_id = gcs_client_->client_table().GetLocalClientId();
SchedulingResources &new_local_resources = cluster_resource_map_[local_client_id];
// SpillOver locally to figure out which infeasible tasks can be placed now
std::vector<TaskID> decision = scheduling_policy_.SpillOver(new_local_resources);
std::unordered_set<TaskID> local_task_ids(decision.begin(), decision.end());
// 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);
}
}
}
void NodeManager::HeartbeatAdded(const ClientID &client_id,
const HeartbeatTableData &heartbeat_data) {
// Locate the client id in remote client table and update available resources based on
// the received heartbeat information.
auto it = 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 &remote_resources = it->second;
ResourceSet remote_total(VectorFromProtobuf(heartbeat_data.resources_total_label()),
VectorFromProtobuf(heartbeat_data.resources_total_capacity()));
ResourceSet remote_available(
VectorFromProtobuf(heartbeat_data.resources_available_label()),
VectorFromProtobuf(heartbeat_data.resources_available_capacity()));
ResourceSet remote_load(VectorFromProtobuf(heartbeat_data.resource_load_label()),
VectorFromProtobuf(heartbeat_data.resource_load_capacity()));
// TODO(atumanov): assert that the load is a non-empty ResourceSet.
remote_resources.SetAvailableResources(std::move(remote_available));
// Extract the load information and save it locally.
remote_resources.SetLoadResources(std::move(remote_load));
if (new_scheduler_enabled_ && client_id != client_id_) {
new_resource_scheduler_->AddOrUpdateNode(client_id.Binary(),
remote_total.GetResourceMap(),
remote_available.GetResourceMap());
NewSchedulerSchedulePendingTasks();
return;
}
// Extract decision for this raylet.
auto decision = scheduling_policy_.SpillOver(remote_resources);
std::unordered_set<TaskID> local_task_ids;
for (const auto &task_id : decision) {
// (See design_docs/task_states.rst for the state transition diagram.)
Task task;
TaskState state;
if (!local_queues_.RemoveTask(task_id, &task, &state)) {
return;
}
// Since we are spilling back from the ready and waiting queues, we need
// to unsubscribe the dependencies.
if (state != TaskState::INFEASIBLE) {
// Don't unsubscribe for infeasible tasks because we never subscribed in
// the first place.
RAY_CHECK(task_dependency_manager_.UnsubscribeGetDependencies(task_id));
}
// Attempt to forward the task. If this fails to forward the task,
// the task will be resubmit locally.
ForwardTaskOrResubmit(task, client_id);
}
}
void NodeManager::HeartbeatBatchAdded(const HeartbeatBatchTableData &heartbeat_batch) {
const ClientID &local_client_id = gcs_client_->client_table().GetLocalClientId();
// Update load information provided by each heartbeat.
// TODO(edoakes): this isn't currently used, but will be used to refresh the LRU
// cache in the object store.
std::unordered_set<ObjectID> active_object_ids;
for (const auto &heartbeat_data : heartbeat_batch.batch()) {
for (int i = 0; i < heartbeat_data.active_object_id_size(); i++) {
active_object_ids.insert(ObjectID::FromBinary(heartbeat_data.active_object_id(i)));
}
const ClientID &client_id = ClientID::FromBinary(heartbeat_data.client_id());
if (client_id == local_client_id) {
// Skip heartbeats from self.
continue;
}
HeartbeatAdded(client_id, heartbeat_data);
}
RAY_LOG(DEBUG) << "Total active object IDs received: " << active_object_ids.size();
// Refresh the active object IDs in plasma to prevent them from being evicted.
std::vector<plasma::ObjectID> plasma_ids;
plasma_ids.reserve(active_object_ids.size());
for (const ObjectID &object_id : active_object_ids) {
plasma_ids.push_back(object_id.ToPlasmaId());
}
if (!store_client_.Refresh(plasma_ids).ok()) {
RAY_LOG(WARNING) << "Failed to refresh active object IDs in plasma.";
}
}
void NodeManager::HandleActorStateTransition(const ActorID &actor_id,
ActorRegistration &&actor_registration) {
// Update local registry.
auto it = actor_registry_.find(actor_id);
if (it == actor_registry_.end()) {
it = actor_registry_.emplace(actor_id, actor_registration).first;
} else {
// Only process the state transition if it is to a later state than ours.
if (actor_registration.GetState() > it->second.GetState() &&
actor_registration.GetRemainingReconstructions() ==
it->second.GetRemainingReconstructions()) {
// The new state is later than ours if it is about the same lifetime, but
// a greater state.
it->second = actor_registration;
} else if (actor_registration.GetRemainingReconstructions() <
it->second.GetRemainingReconstructions()) {
// The new state is also later than ours it is about a later lifetime of
// the actor.
it->second = actor_registration;
} else {
// Our state is already at or past the update, so skip the update.
return;
}
}
RAY_LOG(DEBUG) << "Actor notification received: actor_id = " << actor_id
<< ", node_manager_id = " << actor_registration.GetNodeManagerId()
<< ", state = "
<< ActorTableData::ActorState_Name(actor_registration.GetState())
<< ", remaining_reconstructions = "
<< actor_registration.GetRemainingReconstructions();
if (actor_registration.GetState() == ActorTableData::ALIVE) {
// The actor is now alive (created for the first time or reconstructed). We can
// stop listening for the actor creation task. This is needed because we use
// `ListenAndMaybeReconstruct` to reconstruct the actor.
reconstruction_policy_.Cancel(actor_registration.GetActorCreationDependency());
// 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_.GetTasks(TaskState::WAITING_FOR_ACTOR_CREATION);
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) {
// Maintain the invariant that if a task is in the
// MethodsWaitingForActorCreation queue, then it is subscribed to its
// respective actor creation task. Since the actor location is now known,
// we can remove the task from the queue and forget its dependency on the
// actor creation task.
RAY_CHECK(task_dependency_manager_.UnsubscribeGetDependencies(
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());
}
} else if (actor_registration.GetState() == ActorTableData::DEAD) {
// When an actor dies, loop over all of the queued tasks for that actor
// and treat them as failed.
auto tasks_to_remove = local_queues_.GetTaskIdsForActor(actor_id);
auto removed_tasks = local_queues_.RemoveTasks(tasks_to_remove);
for (auto const &task : removed_tasks) {
TreatTaskAsFailed(task, ErrorType::ACTOR_DIED);
}
} else {
RAY_CHECK(actor_registration.GetState() == ActorTableData::RECONSTRUCTING);
RAY_LOG(DEBUG) << "Actor is being reconstructed: " << actor_id;
// The actor is dead and needs reconstruction. Attempting to reconstruct its
// creation task.
reconstruction_policy_.ListenAndMaybeReconstruct(
actor_registration.GetActorCreationDependency());
// When an actor fails but can be reconstructed, resubmit all of the queued
// tasks for that actor. This will mark the tasks as waiting for actor
// creation.
auto tasks_to_remove = local_queues_.GetTaskIdsForActor(actor_id);
auto removed_tasks = local_queues_.RemoveTasks(tasks_to_remove);
for (auto const &task : removed_tasks) {
SubmitTask(task, Lineage());
}
}
}
void NodeManager::ProcessNewClient(LocalClientConnection &client) {
// The new client is a worker, so begin listening for messages.
client.ProcessMessages();
}
// A helper function to create a mapping from task scheduling class to
// tasks with that class from a given list of tasks.
std::unordered_map<SchedulingClass, ordered_set<TaskID>> MakeTasksByClass(
const std::vector<Task> &tasks) {
std::unordered_map<SchedulingClass, ordered_set<TaskID>> result;
for (const auto &task : tasks) {
auto spec = task.GetTaskSpecification();
result[spec.GetSchedulingClass()].push_back(spec.TaskId());
}
return result;
}
void NodeManager::DispatchTasks(
const std::unordered_map<SchedulingClass, ordered_set<TaskID>> &tasks_by_class) {
// Dispatch tasks in priority order by class. This avoids starvation problems where
// one class of tasks become stuck behind others in the queue, causing Ray to start
// many workers. See #3644 for a more detailed description of this issue.
std::vector<const std::pair<const SchedulingClass, ordered_set<TaskID>> *> fair_order;
for (auto &it : tasks_by_class) {
fair_order.emplace_back(&it);
}
// Prioritize classes that have fewer currently running tasks. Note that we only
// sort once per round of task dispatch, which is less fair then it could be, but
// is simpler and faster.
if (fair_queueing_enabled_) {
std::sort(
std::begin(fair_order), std::end(fair_order),
[this](const std::pair<const SchedulingClass, ordered_set<ray::TaskID>> *a,
const std::pair<const SchedulingClass, ordered_set<ray::TaskID>> *b) {
return local_queues_.NumRunning(a->first) < local_queues_.NumRunning(b->first);
});
}
std::vector<std::function<void()>> post_assign_callbacks;
// Approximate fair round robin between classes.
for (const auto &it : fair_order) {
const auto &task_resources =
TaskSpecification::GetSchedulingClassDescriptor(it->first).first;
// FIFO order within each class.
for (const auto &task_id : it->second) {
const auto &task = local_queues_.GetTaskOfState(task_id, TaskState::READY);
if (!local_available_resources_.Contains(task_resources)) {
// All the tasks in it.second have the same resource shape, so
// once the first task is not feasible, we can break out of this loop
break;
}
// Try to get an idle worker to execute this task. If nullptr, there
// aren't any available workers so we can't assign the task.
std::shared_ptr<Worker> worker =
worker_pool_.PopWorker(task.GetTaskSpecification());
if (worker != nullptr) {
AssignTask(worker, task, &post_assign_callbacks);
}
}
}
// Call the callbacks from the AssignTask calls above. These need to be called
// after the above loop, as they may alter the scheduling queues and invalidate
// the loop iterator.
for (auto &func : post_assign_callbacks) {
func();
}
}
void NodeManager::ProcessClientMessage(
const std::shared_ptr<LocalClientConnection> &client, int64_t message_type,
const uint8_t *message_data) {
auto registered_worker = worker_pool_.GetRegisteredWorker(client);
auto message_type_value = static_cast<protocol::MessageType>(message_type);
RAY_LOG(DEBUG) << "[Worker] Message "
<< protocol::EnumNameMessageType(message_type_value) << "("
<< message_type << ") from worker with PID "
<< (registered_worker ? std::to_string(registered_worker->Pid())
: "nil");
if (registered_worker && registered_worker->IsDead()) {
// For a worker that is marked as dead (because the job has died already),
// all the messages are ignored except DisconnectClient.
if ((message_type_value != protocol::MessageType::DisconnectClient) &&
(message_type_value != protocol::MessageType::IntentionalDisconnectClient)) {
// Listen for more messages.
client->ProcessMessages();
return;
}
}
switch (message_type_value) {
case protocol::MessageType::RegisterClientRequest: {
ProcessRegisterClientRequestMessage(client, message_data);
} break;
case protocol::MessageType::TaskDone: {
HandleWorkerAvailable(client);
} break;
case protocol::MessageType::DisconnectClient: {
ProcessDisconnectClientMessage(client);
// We don't need to receive future messages from this client,
// because it's already disconnected.
return;
} break;
case protocol::MessageType::IntentionalDisconnectClient: {
ProcessDisconnectClientMessage(client, /* intentional_disconnect = */ true);
// We don't need to receive future messages from this client,
// because it's already disconnected.
return;
} break;
case protocol::MessageType::SubmitTask: {
// For tasks submitted via the raylet path, we must make sure to order the
// task submission so that tasks are always submitted after the tasks that
// they depend on.
ProcessSubmitTaskMessage(message_data);
} break;
case protocol::MessageType::SetResourceRequest: {
ProcessSetResourceRequest(client, message_data);
} break;
case protocol::MessageType::FetchOrReconstruct: {
ProcessFetchOrReconstructMessage(client, message_data);
} break;
case protocol::MessageType::NotifyDirectCallTaskBlocked: {
std::shared_ptr<Worker> worker = worker_pool_.GetRegisteredWorker(client);
HandleDirectCallTaskBlocked(worker);
} break;
case protocol::MessageType::NotifyDirectCallTaskUnblocked: {
std::shared_ptr<Worker> worker = worker_pool_.GetRegisteredWorker(client);
HandleDirectCallTaskUnblocked(worker);
} break;
case protocol::MessageType::NotifyUnblocked: {
auto message = flatbuffers::GetRoot<protocol::NotifyUnblocked>(message_data);
AsyncResolveObjectsFinish(client, from_flatbuf<TaskID>(*message->task_id()),
/*was_blocked*/ true);
} break;
case protocol::MessageType::WaitRequest: {
ProcessWaitRequestMessage(client, message_data);
} break;
case protocol::MessageType::WaitForDirectActorCallArgsRequest: {
ProcessWaitForDirectActorCallArgsRequestMessage(client, message_data);
} break;
case protocol::MessageType::PushErrorRequest: {
ProcessPushErrorRequestMessage(message_data);
} break;
case protocol::MessageType::PushProfileEventsRequest: {
auto fbs_message = flatbuffers::GetRoot<flatbuffers::String>(message_data);
rpc::ProfileTableData profile_table_data;
RAY_CHECK(
profile_table_data.ParseFromArray(fbs_message->data(), fbs_message->size()));
RAY_CHECK_OK(gcs_client_->profile_table().AddProfileEventBatch(profile_table_data));
} break;
case protocol::MessageType::FreeObjectsInObjectStoreRequest: {
auto message = flatbuffers::GetRoot<protocol::FreeObjectsRequest>(message_data);
std::vector<ObjectID> object_ids = from_flatbuf<ObjectID>(*message->object_ids());
// Clean up objects from the object store.
object_manager_.FreeObjects(object_ids, message->local_only());
if (message->delete_creating_tasks()) {
// Clean up their creating tasks from GCS.
std::vector<TaskID> creating_task_ids;
for (const auto &object_id : object_ids) {
creating_task_ids.push_back(object_id.TaskId());
}
gcs_client_->raylet_task_table().Delete(JobID::Nil(), creating_task_ids);
}
} break;
case protocol::MessageType::PrepareActorCheckpointRequest: {
ProcessPrepareActorCheckpointRequest(client, message_data);
} break;
case protocol::MessageType::NotifyActorResumedFromCheckpoint: {
ProcessNotifyActorResumedFromCheckpoint(message_data);
} break;