-
Notifications
You must be signed in to change notification settings - Fork 5.3k
/
node_manager.cc
2018 lines (1842 loc) · 93 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 <fstream>
#include "arrow/util/logging.h"
#include "ray/common/common_protocol.h"
#include "ray/id.h"
#include "ray/raylet/format/node_manager_generated.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::ActorHandleID &actor_handle_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_handle_id);
if (frontier_entry != frontier.end()) {
expected_task_counter = frontier_entry->second.task_counter;
}
return expected_task_counter;
};
} // 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,
std::shared_ptr<ObjectDirectoryInterface> object_directory)
: 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),
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.num_workers_per_process,
config.maximum_startup_concurrency, config.worker_commands),
scheduling_policy_(local_queues_),
reconstruction_policy_(
io_service_,
[this](const TaskID &task_id, bool return_values_lost) {
HandleTaskReconstruction(task_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),
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 object_manager::protocol::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()));
}
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 to handle actor notifications.
auto actor_notification_callback = [this](gcs::AsyncGcsClient *client,
const ActorID &actor_id,
const std::vector<ActorTableDataT> &data) {
if (!data.empty()) {
// We only need the last entry, because it represents the latest state of
// this actor.
HandleActorStateTransition(actor_id, data.back());
}
};
RAY_RETURN_NOT_OK(gcs_client_->actor_table().Subscribe(
UniqueID::nil(), UniqueID::nil(), actor_notification_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 heartbeat batches from the monitor.
const auto &heartbeat_batch_added = [this](
gcs::AsyncGcsClient *client, const ClientID &id,
const HeartbeatBatchTableDataT &heartbeat_batch) {
HeartbeatBatchAdded(heartbeat_batch);
};
RAY_RETURN_NOT_OK(gcs_client_->heartbeat_batch_table().Subscribe(
UniqueID::nil(), UniqueID::nil(), heartbeat_batch_added, nullptr,
[](gcs::AsyncGcsClient *client) {
RAY_LOG(DEBUG) << "Heartbeat batch table subscription done.";
}));
// 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();
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::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) {
auto driver_id = UniqueID::from_binary(entry.driver_id);
auto workers = worker_pool_.GetWorkersRunningTasksForDriver(driver_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) {
// 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 driver 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.
CleanUpTasksForDeadDriver(driver_id);
}
}
}
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>();
const auto &my_client_id = gcs_client_->client_table().GetLocalClientId();
SchedulingResources &local_resources = cluster_resource_map_[my_client_id];
heartbeat_data->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.
RAY_LOG(DEBUG) << "[Heartbeat] resources available: "
<< local_resources.GetAvailableResources().ToString();
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);
}
local_resources.SetLoadResources(local_queues_.GetResourceLoad());
for (const auto &resource_pair : local_resources.GetLoadResources().GetResourceMap()) {
heartbeat_data->resource_load_label.push_back(resource_pair.first);
heartbeat_data->resource_load_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);
if (debug_dump_period_ > 0 &&
static_cast<int64_t>(now_ms - last_debug_dump_at_ms_) > debug_dump_period_) {
DumpDebugState();
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();
});
}
void NodeManager::GetObjectManagerProfileInfo() {
int64_t start_time_ms = current_time_ms();
auto profile_info = object_manager_.GetAndResetProfilingInfo();
if (profile_info.profile_events.size() > 0) {
flatbuffers::FlatBufferBuilder fbb;
auto message = CreateProfileTableData(fbb, &profile_info);
fbb.Finish(message);
auto profile_message = flatbuffers::GetRoot<ProfileTableData>(fbb.GetBufferPointer());
RAY_CHECK_OK(gcs_client_->profile_table().AddProfileEventBatch(*profile_message));
}
// 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 ClientTableDataT &client_data) {
const 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] = initial_config_.resource_config;
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;
}
// Establish a new NodeManager connection to this GCS client.
RAY_LOG(DEBUG) << "[ClientAdded] Trying to connect to client " << client_id << " at "
<< client_data.node_manager_address << ":"
<< client_data.node_manager_port;
boost::asio::ip::tcp::socket socket(io_service_);
auto status =
TcpConnect(socket, client_data.node_manager_address, client_data.node_manager_port);
// A disconnected client has 2 entries in the client table (one for being
// inserted and one for being removed). When a new raylet starts, ClientAdded
// will be called with the disconnected client's first entry, which will cause
// IOError and "Connection refused".
if (!status.ok()) {
RAY_LOG(WARNING) << "Failed to connect to client " << client_id
<< " in ClientAdded. TcpConnect returned status: "
<< status.ToString() << ". This may be caused by "
<< "trying to connect to a node manager that has failed.";
return;
}
// The client is connected.
auto server_conn = TcpServerConnection::Create(std::move(socket));
remote_server_connections_.emplace(client_id, std::move(server_conn));
ResourceSet resources_total(client_data.resources_total_label,
client_data.resources_total_capacity);
cluster_resource_map_.emplace(client_id, SchedulingResources(resources_total));
}
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.
const auto connection_entry = remote_server_connections_.find(client_id);
if (connection_entry != remote_server_connections_.end()) {
connection_entry->second->Close();
remote_server_connections_.erase(connection_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() == ActorState::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);
}
void NodeManager::HeartbeatAdded(const ClientID &client_id,
const HeartbeatTableDataT &heartbeat_data) {
RAY_LOG(DEBUG) << "[HeartbeatAdded]: received heartbeat from client id " << client_id;
// 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_available(heartbeat_data.resources_available_label,
heartbeat_data.resources_available_capacity);
ResourceSet remote_load(heartbeat_data.resource_load_label,
heartbeat_data.resource_load_capacity);
// TODO(atumanov): assert that the load is a non-empty ResourceSet.
RAY_LOG(DEBUG) << "[HeartbeatAdded]: received load: " << remote_load.ToString();
remote_resources.SetAvailableResources(std::move(remote_available));
// Extract the load information and save it locally.
remote_resources.SetLoadResources(std::move(remote_load));
// Extract decision for this local scheduler.
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.)
TaskState state;
const auto task = local_queues_.RemoveTask(task_id, &state);
// 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_.UnsubscribeDependencies(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 HeartbeatBatchTableDataT &heartbeat_batch) {
const ClientID &local_client_id = gcs_client_->client_table().GetLocalClientId();
// Update load information provided by each heartbeat.
for (const auto &heartbeat_data : heartbeat_batch.batch) {
const ClientID &client_id = ClientID::from_binary(heartbeat_data->client_id);
if (client_id == local_client_id) {
// Skip heartbeats from self.
continue;
}
HeartbeatAdded(client_id, *heartbeat_data);
}
}
void NodeManager::PublishActorStateTransition(
const ActorID &actor_id, const ActorTableDataT &data,
const ray::gcs::ActorTable::WriteCallback &failure_callback) {
// Copy the actor notification data.
auto actor_notification = std::make_shared<ActorTableDataT>(data);
// The actor log starts with an ALIVE entry. This is followed by 0 to N pairs
// of (RECONSTRUCTING, ALIVE) entries, where N is the maximum number of
// reconstructions. This is followed optionally by a DEAD entry.
int log_length = 2 * (actor_notification->max_reconstructions -
actor_notification->remaining_reconstructions);
if (actor_notification->state != ActorState::ALIVE) {
// RECONSTRUCTING or DEAD entries have an odd index.
log_length += 1;
}
RAY_CHECK_OK(gcs_client_->actor_table().AppendAt(
JobID::nil(), actor_id, actor_notification, nullptr, failure_callback, log_length));
}
void NodeManager::HandleActorStateTransition(const ActorID &actor_id,
const ActorTableDataT &data) {
ActorRegistration actor_registration(data);
RAY_LOG(DEBUG) << "Actor notification received: actor_id = " << actor_id
<< ", node_manager_id = " << actor_registration.GetNodeManagerId()
<< ", state = " << EnumNameActorState(actor_registration.GetState())
<< ", remaining_reconstructions = "
<< actor_registration.GetRemainingReconstructions();
// 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;
}
}
if (actor_registration.GetState() == ActorState::ALIVE) {
// 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) {
if (!lineage_cache_.RemoveWaitingTask(method.GetTaskSpecification().TaskId())) {
RAY_LOG(WARNING) << "Task " << method.GetTaskSpecification().TaskId()
<< " already removed from the lineage cache. This is most "
"likely due to reconstruction.";
}
// 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_.UnsubscribeDependencies(
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() == ActorState::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);
}
} else {
RAY_CHECK(actor_registration.GetState() == ActorState::RECONSTRUCTING);
RAY_LOG(DEBUG) << "Actor is being reconstructed: " << actor_id;
// 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::CleanUpTasksForDeadDriver(const DriverID &driver_id) {
auto tasks_to_remove = local_queues_.GetTaskIdsForDriver(driver_id);
local_queues_.RemoveTasks(tasks_to_remove);
task_dependency_manager_.RemoveTasksAndRelatedObjects(tasks_to_remove);
}
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 resource shapes to
// tasks with that resource shape from a given list of tasks.
std::unordered_map<ResourceSet, ordered_set<TaskID>> MakeTasksWithResources(
const std::vector<Task> &tasks) {
std::unordered_map<ResourceSet, ordered_set<TaskID>> result;
for (const auto &task : tasks) {
auto spec = task.GetTaskSpecification();
result[spec.GetRequiredResources()].push_back(spec.TaskId());
}
return result;
}
void NodeManager::DispatchTasks(
const std::unordered_map<ResourceSet, ordered_set<TaskID>> &tasks_with_resources) {
std::unordered_set<TaskID> removed_task_ids;
for (const auto &it : tasks_with_resources) {
for (const auto &task_id : it.second) {
const auto &task = local_queues_.GetReadyQueue().GetTask(task_id);
const auto &task_resources = task.GetTaskSpecification().GetRequiredResources();
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;
}
if (AssignTask(task)) {
removed_task_ids.insert(task_id);
}
}
}
local_queues_.RemoveTasks(removed_task_ids);
}
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) << "Message of " << protocol::EnumNameMessageType(message_type_value)
<< "(" << message_type << ")";
if (registered_worker && registered_worker->IsDead()) {
// For a worker that is marked as dead (because the driver 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::GetTask: {
ProcessGetTaskMessage(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: {
ProcessSubmitTaskMessage(message_data);
} break;
case protocol::MessageType::FetchOrReconstruct: {
ProcessFetchOrReconstructMessage(client, message_data);
} break;
case protocol::MessageType::NotifyUnblocked: {
auto message = flatbuffers::GetRoot<protocol::NotifyUnblocked>(message_data);
HandleTaskUnblocked(client, from_flatbuf(*message->task_id()));
} break;
case protocol::MessageType::WaitRequest: {
ProcessWaitRequestMessage(client, message_data);
} break;
case protocol::MessageType::PushErrorRequest: {
ProcessPushErrorRequestMessage(message_data);
} break;
case protocol::MessageType::PushProfileEventsRequest: {
auto message = flatbuffers::GetRoot<ProfileTableData>(message_data);
RAY_CHECK_OK(gcs_client_->profile_table().AddProfileEventBatch(*message));
} break;
case protocol::MessageType::FreeObjectsInObjectStoreRequest: {
auto message = flatbuffers::GetRoot<protocol::FreeObjectsRequest>(message_data);
std::vector<ObjectID> object_ids = from_flatbuf(*message->object_ids());
object_manager_.FreeObjects(object_ids, message->local_only());
} break;
default:
RAY_LOG(FATAL) << "Received unexpected message type " << message_type;
}
// Listen for more messages.
client->ProcessMessages();
}
void NodeManager::ProcessRegisterClientRequestMessage(
const std::shared_ptr<LocalClientConnection> &client, const uint8_t *message_data) {
auto message = flatbuffers::GetRoot<protocol::RegisterClientRequest>(message_data);
client->SetClientID(from_flatbuf(*message->client_id()));
auto worker =
std::make_shared<Worker>(message->worker_pid(), message->language(), client);
if (message->is_worker()) {
// Register the new worker.
worker_pool_.RegisterWorker(std::move(worker));
DispatchTasks(local_queues_.GetReadyQueue().GetTasksWithResources());
} else {
// Register the new driver. Note that here the driver_id in RegisterClientRequest
// message is actually the ID of the driver task, while client_id represents the
// real driver ID, which can associate all the tasks/actors for a given driver,
// which is set to the worker ID.
const JobID driver_task_id = from_flatbuf(*message->driver_id());
worker->AssignTaskId(driver_task_id);
worker->AssignDriverId(from_flatbuf(*message->client_id()));
worker_pool_.RegisterDriver(std::move(worker));
local_queues_.AddDriverTaskId(driver_task_id);
}
}
void NodeManager::HandleDisconnectedActor(const ActorID &actor_id, bool was_local,
bool intentional_disconnect) {
auto actor_entry = actor_registry_.find(actor_id);
RAY_CHECK(actor_entry != actor_registry_.end());
auto &actor_registration = actor_entry->second;
RAY_LOG(DEBUG) << "The actor with ID " << actor_id << " died "
<< (intentional_disconnect ? "intentionally" : "unintentionally")
<< ", remaining reconstructions = "
<< actor_registration.GetRemainingReconstructions();
// Check if this actor needs to be reconstructed.
ActorState new_state =
actor_registration.GetRemainingReconstructions() > 0 && !intentional_disconnect
? ActorState::RECONSTRUCTING
: ActorState::DEAD;
if (was_local) {
// Clean up the dummy objects from this actor.
RAY_LOG(DEBUG) << "Removing dummy objects for actor: " << actor_id;
for (auto &dummy_object_pair : actor_entry->second.GetDummyObjects()) {
HandleObjectMissing(dummy_object_pair.first);
}
}
// Update the actor's state.
ActorTableDataT new_actor_data = actor_entry->second.GetTableData();
new_actor_data.state = new_state;
if (was_local) {
// If the actor was local, immediately update the state in actor registry.
// So if we receive any actor tasks before we receive GCS notification,
// these tasks can be correctly routed to the `MethodsWaitingForActorCreation` queue,
// instead of being assigned to the dead actor.
HandleActorStateTransition(actor_id, new_actor_data);
}
ray::gcs::ActorTable::WriteCallback failure_callback = nullptr;
if (was_local) {
failure_callback = [](gcs::AsyncGcsClient *client, const ActorID &id,
const ActorTableDataT &data) {
// If the disconnected actor was local, only this node will try to update actor
// state. So the update shouldn't fail.
RAY_LOG(FATAL) << "Failed to update state for actor " << id;
};
}
PublishActorStateTransition(actor_id, new_actor_data, failure_callback);
}
void NodeManager::ProcessGetTaskMessage(
const std::shared_ptr<LocalClientConnection> &client) {
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));
// Local resource availability changed: invoke scheduling policy for local node.
const ClientID &local_client_id = gcs_client_->client_table().GetLocalClientId();
cluster_resource_map_[local_client_id].SetLoadResources(
local_queues_.GetResourceLoad());
// Call task dispatch to assign work to the new worker.
DispatchTasks(local_queues_.GetReadyQueue().GetTasksWithResources());
}
void NodeManager::ProcessDisconnectClientMessage(
const std::shared_ptr<LocalClientConnection> &client, bool intentional_disconnect) {
std::shared_ptr<Worker> worker = worker_pool_.GetRegisteredWorker(client);
bool is_worker = false, is_driver = false;
if (worker) {
// The client is a worker.
is_worker = true;
} else {
worker = worker_pool_.GetRegisteredDriver(client);
if (worker) {
// The client is a driver.
is_driver = true;
} else {
RAY_LOG(INFO) << "Ignoring client disconnect because the client has already "
<< "been disconnected.";
}
}
RAY_CHECK(!(is_worker && is_driver));
// If the client has any blocked tasks, mark them as unblocked. In
// particular, we are no longer waiting for their dependencies.
if (worker) {
while (!worker->GetBlockedTaskIds().empty()) {
// NOTE(swang): HandleTaskUnblocked will modify the worker, so it is
// not safe to pass in the iterator directly.
const TaskID task_id = *worker->GetBlockedTaskIds().begin();
HandleTaskUnblocked(client, task_id);
}
}
// Remove the dead client from the pool and stop listening for messages.
if (is_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() && !worker->IsDead()) {
// If the worker was killed intentionally, e.g., when the driver that created
// the task that this worker is currently executing exits, the task for this
// worker has already been removed from queue, so the following are skipped.
const Task &task = local_queues_.RemoveTask(task_id);
// Handle the task failure in order to raise an exception in the
// application.
TreatTaskAsFailed(task);
const JobID &job_id = worker->GetAssignedDriverId();
if (!intentional_disconnect) {
// 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()));
}
}
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()) {
RAY_LOG(DEBUG) << "The actor with ID " << actor_id << " died on "
<< gcs_client_->client_table().GetLocalClientId();
HandleDisconnectedActor(actor_id, /*was_local=*/true, intentional_disconnect);
}
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();
RAY_LOG(DEBUG) << "Worker (pid=" << worker->Pid() << ") is disconnected. "
<< "driver_id: " << worker->GetAssignedDriverId();
// Since some resources may have been released, we can try to dispatch more tasks.
DispatchTasks(local_queues_.GetReadyQueue().GetTasksWithResources());
} else if (is_driver) {
// The client is a driver.
RAY_CHECK_OK(gcs_client_->driver_table().AppendDriverData(client->GetClientId(),
/*is_dead=*/true));
auto driver_id = worker->GetAssignedTaskId();
RAY_CHECK(!driver_id.is_nil());
local_queues_.RemoveDriverTaskId(driver_id);
worker_pool_.DisconnectDriver(worker);
RAY_LOG(DEBUG) << "Driver (pid=" << worker->Pid() << ") is disconnected. "
<< "driver_id: " << worker->GetAssignedDriverId();
}
// TODO(rkn): Tell the object manager that this client has disconnected so
// that it can clean up the wait requests for this client. Currently I think
// these can be leaked.
}
void NodeManager::ProcessSubmitTaskMessage(const uint8_t *message_data) {
// 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());
}
void NodeManager::ProcessFetchOrReconstructMessage(
const std::shared_ptr<LocalClientConnection> &client, const uint8_t *message_data) {
auto message = flatbuffers::GetRoot<protocol::FetchOrReconstruct>(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 (message->fetch_only()) {
// If only a fetch is required, then do not subscribe to the
// dependencies to the task dependency manager.
if (!task_dependency_manager_.CheckObjectLocal(object_id)) {
// Fetch the object if it's not already local.
RAY_CHECK_OK(object_manager_.Pull(object_id));
}
} else {
// If reconstruction is also required, then add any requested 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()) {
const TaskID task_id = from_flatbuf(*message->task_id());
HandleTaskBlocked(client, required_object_ids, task_id);
}
}
void NodeManager::ProcessWaitRequestMessage(
const std::shared_ptr<LocalClientConnection> &client, const uint8_t *message_data) {
// 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();
std::vector<ObjectID> required_object_ids;
for (auto const &object_id : object_ids) {
if (!task_dependency_manager_.CheckObjectLocal(object_id)) {
// 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);
}
}
const TaskID ¤t_task_id = from_flatbuf(*message->task_id());
bool client_blocked = !required_object_ids.empty();
if (client_blocked) {
HandleTaskBlocked(client, required_object_ids, current_task_id);
}
ray::Status status = object_manager_.Wait(
object_ids, wait_ms, num_required_objects, wait_local,
[this, client_blocked, client, current_task_id](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);
auto status =
client->WriteMessage(static_cast<int64_t>(protocol::MessageType::WaitReply),
fbb.GetSize(), fbb.GetBufferPointer());
if (status.ok()) {
// The client is unblocked now because the wait call has returned.
if (client_blocked) {
HandleTaskUnblocked(client, current_task_id);
}
} else {
// We failed to write to the client, so disconnect the client.
RAY_LOG(WARNING)
<< "Failed to send WaitReply to client, so disconnecting client";
// We failed to send the reply to the client, so disconnect the worker.
ProcessDisconnectClientMessage(client);
}
});
RAY_CHECK_OK(status);
}
void NodeManager::ProcessPushErrorRequestMessage(const uint8_t *message_data) {
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));
}